Merge branch 'gerrit/cheshire-cat'
Change-Id: Ie27e28a9849ed5cf750a592dd77115fa2f7fb212
diff --git a/asterixdb/asterix-active/pom.xml b/asterixdb/asterix-active/pom.xml
index 5190f9d..d47078a 100644
--- a/asterixdb/asterix-active/pom.xml
+++ b/asterixdb/asterix-active/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-active</artifactId>
<properties>
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index cf5802f..18c3437 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-algebra</artifactId>
@@ -231,5 +231,10 @@
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-fuzzyjoin</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
index 6578c9c..135c3e3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/CommitOperator.java
@@ -44,6 +44,10 @@
this.isSink = isSink;
}
+ public List<LogicalVariable> getPrimaryKeyLogicalVars() {
+ return this.primaryKeyLogicalVars;
+ }
+
@Override
public boolean isMap() {
return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 8c2c1df..ed8e9bf 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -156,7 +156,7 @@
return false;
}
Index searchIndex = ((DataSourceIndex) idx).getIndex();
- int numberOfKeyFields = searchIndex.getKeyFieldNames().size();
+ int numberOfKeyFields = ((Index.ValueIndexDetails) searchIndex.getIndexDetails()).getKeyFieldNames().size();
if (lowKeyVarList.size() != numberOfKeyFields || highKeyVarList.size() != numberOfKeyFields) {
return false;
@@ -186,7 +186,8 @@
// If this is a composite primary index, then all of the keys should be provided.
Index searchIndex = ((DataSourceIndex) idx).getIndex();
- int numberOfKeyFields = searchIndex.getKeyFieldNames().size();
+ int numberOfKeyFields =
+ ((Index.ValueIndexDetails) searchIndex.getIndexDetails()).getKeyFieldNames().size();
if (numberOfKeyFields < 2
|| (lowKeyVarList.size() == numberOfKeyFields && highKeyVarList.size() == numberOfKeyFields)) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 4ad7d3f..532fb43 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -26,6 +26,7 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.FullTextUtil;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -61,6 +62,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
/**
@@ -170,15 +172,20 @@
InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, secondaryIndex);
IBinaryTokenizerFactory queryTokenizerFactory =
InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, secondaryIndex);
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+ FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider, secondaryIndex.getDataverseName(),
+ ((Index.TextIndexDetails) secondaryIndex.getIndexDetails()).getFullTextConfigName());
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondarySplitsAndConstraint.first);
- LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
- jobSpec, outputRecDesc, queryField, dataflowHelperFactory, queryTokenizerFactory, searchModifierFactory,
- retainInput, retainMissing, context.getMissingWriterFactory(),
- dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
- IndexOperation.SEARCH, null),
- minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
- propagateIndexFilter, frameLimit);
+
+ LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp =
+ new LSMInvertedIndexSearchOperatorDescriptor(jobSpec, outputRecDesc, queryField, dataflowHelperFactory,
+ queryTokenizerFactory, fullTextConfigEvaluatorFactory, searchModifierFactory, retainInput,
+ retainMissing, context.getMissingWriterFactory(),
+ dataset.getSearchCallbackFactory(metadataProvider.getStorageComponentProvider(), secondaryIndex,
+ IndexOperation.SEARCH, null),
+ minFilterFieldIndexes, maxFilterFieldIndexes, isFullTextSearchQuery, numPrimaryKeys,
+ propagateIndexFilter, frameLimit);
return new Pair<>(invIndexSearchOp, secondarySplitsAndConstraint.second);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 2a662d0..b1d0b47 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -137,9 +137,14 @@
IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
throws AlgebricksException {
IScalarEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
- IFunctionDescriptor fd = expr.getFunctionInfo() instanceof IExternalFunctionInfo
- ? ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context)
- : resolveFunction(expr, env, context);
+ IFunctionDescriptor fd;
+ if (expr.getFunctionInfo() instanceof IExternalFunctionInfo) {
+ // Expr is an external function
+ fd = ExternalFunctionDescriptorProvider.resolveExternalFunction(expr, env, context);
+ } else {
+ // Expr is an internal (built-in) function
+ fd = resolveFunction(expr, env, context);
+ }
return fd.createEvaluatorFactory(args);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index fc52f89..b6e287b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -47,7 +47,7 @@
import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
import org.apache.asterix.optimizer.rules.FindDataSourcesRule;
import org.apache.asterix.optimizer.rules.FixReplicateOperatorOutputsRule;
-import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckRule;
+import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckAndSetRule;
import org.apache.asterix.optimizer.rules.FuzzyEqRule;
import org.apache.asterix.optimizer.rules.InjectTypeCastForFunctionArgumentsRule;
import org.apache.asterix.optimizer.rules.InjectTypeCastForUnionRule;
@@ -177,7 +177,7 @@
}
public static final List<IAlgebraicRewriteRule> buildFulltextContainsRuleCollection() {
- return Collections.singletonList(new FullTextContainsParameterCheckRule());
+ return Collections.singletonList(new FullTextContainsParameterCheckAndSetRule());
}
public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection(ICcApplicationContext appCtx) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java
new file mode 100644
index 0000000..3e4e563
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckAndSetRule.java
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.utils.FullTextUtil;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+import com.google.common.base.Strings;
+
+/**
+ * Checks whether the given parameters of the ftcontains() function are correct during the compilation,
+ * and fetch the full-text config from metadata which is necessary for the ftcontains() function
+ */
+public class FullTextContainsParameterCheckAndSetRule implements IAlgebraicRewriteRule {
+ // Visitor for checking and transforming ftcontains() expression
+ protected FullTextContainsExpressionVisitor ftcontainsExprVisitor = new FullTextContainsExpressionVisitor();
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+ return false;
+ }
+ if (checkAndSetParameter(opRef, context)) {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Check the correctness of the parameters of the ftcontains(). Also rearrange options as arguments.
+ * The expected form of ftcontains() is ftcontains(expression1, expression2, parameters as a record).
+ *
+ * If ftcontains() has the full-text config argument, this method will also fetch it (FullTextConfigDescriptor) from metadata
+ * and set it in the function expression so that the full-text config can be utilized later at run-time.
+ */
+ private boolean checkAndSetParameter(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ ftcontainsExprVisitor.setContext(context);
+ boolean modified = op.acceptExpressionTransform(ftcontainsExprVisitor);
+ if (modified) {
+ context.addToDontApplySet(this, op);
+ }
+ return modified;
+ }
+
+ /**
+ * This visitor class handles actual checking and transformation.
+ */
+ protected class FullTextContainsExpressionVisitor implements ILogicalExpressionReferenceTransform {
+
+ // the last expression position before the option argument in the arguments array
+ private static final int LAST_EXPRESSION_POS_BEFORE_OPTION = 1;
+ // The number of anticipated arguments for a full-text query when a user doesn't provide any option.
+ private static final int FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS = 2;
+ // The number of anticipated arguments for a full-text query when a user provide option(s) as a record.
+ private static final int FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS = 3;
+
+ private IOptimizationContext context;
+
+ public FullTextContainsExpressionVisitor() {
+ }
+
+ public void setContext(IOptimizationContext context) {
+ this.context = context;
+ }
+
+ @Override
+ public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
+ ILogicalExpression e = exprRef.getValue();
+ switch (e.getExpressionTag()) {
+ case FUNCTION_CALL:
+ return transformFunctionCallExpression((AbstractFunctionCallExpression) e);
+ default:
+ return false;
+ }
+ }
+
+ private boolean transformFunctionCallExpression(AbstractFunctionCallExpression fce) throws AlgebricksException {
+ boolean modified = false;
+ FunctionIdentifier fi = fce.getFunctionIdentifier();
+ if (fi != BuiltinFunctions.FULLTEXT_CONTAINS && fi != BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+ for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
+ if (transform(arg)) {
+ modified = true;
+ }
+ }
+ } else {
+ modified = checkParameterForFuncExpr(fce, fi);
+ }
+
+ return modified;
+ }
+
+ private boolean checkParameterForFuncExpr(AbstractFunctionCallExpression funcExpr, FunctionIdentifier fi)
+ throws AlgebricksException {
+ // Collects the correct number of arguments - it can be 2 if a user doesn't provide any option.
+ int numberOfCorrectArguments = 0;
+ String functionName = "";
+ if (fi == BuiltinFunctions.FULLTEXT_CONTAINS) {
+ numberOfCorrectArguments = FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS;
+ functionName = BuiltinFunctions.FULLTEXT_CONTAINS.getName();
+ } else if (fi == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
+ numberOfCorrectArguments = FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS;
+ functionName = BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION.getName();
+ }
+
+ // If numberOfCorrectArguments is greater than zero, then this is a full-text search query.
+ if (numberOfCorrectArguments > 0) {
+
+ List<Mutable<ILogicalExpression>> oldExprs = funcExpr.getArguments();
+ List<Mutable<ILogicalExpression>> newExprs = new ArrayList<>();
+
+ // The number of parameters should be three: exp1, exp2, and the option
+ if (oldExprs.size() != numberOfCorrectArguments) {
+ throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
+ funcExpr.getSourceLocation(), fi.getName(), oldExprs.size());
+ }
+
+ // The last expression before the option needs to be copied first.
+ for (int i = 0; i <= LAST_EXPRESSION_POS_BEFORE_OPTION; i++) {
+ newExprs.add(new MutableObject<ILogicalExpression>(oldExprs.get(i).getValue()));
+ }
+
+ // Sanity check for the types of the first two parameters
+ checkFirstAndSecondParamter(oldExprs, functionName);
+
+ // We cannot make ftConfigName a class-level variable in the visitor class
+ // because the visitor may be shared between multiple threads and such a variable may be corrupted
+ String ftConfigName = null;
+ // Checks and transforms the actual full-text parameters.
+ if (numberOfCorrectArguments == FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS) {
+ ftConfigName =
+ checkValueForThirdParameterAndGetFullTextConfig(oldExprs.get(2), newExprs, functionName);
+ } else {
+ // no option provided case: sets the default option here.
+ setDefaultValueForThirdParameter(newExprs);
+ }
+
+ MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
+ DataverseName dataverseName = metadataProvider.getDefaultDataverseName();
+ funcExpr.setOpaqueParameters(new Object[] { FullTextUtil
+ .fetchFilterAndCreateConfigEvaluator(metadataProvider, dataverseName, ftConfigName) });
+ // Resets the last argument.
+ funcExpr.getArguments().clear();
+ funcExpr.getArguments().addAll(newExprs);
+
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ * Checks the correctness of the first and second argument. If the argument is a constant, we can check
+ * it now. If the argument is not a constant, we will defer the checking until run-time.
+ */
+ private void checkFirstAndSecondParamter(List<Mutable<ILogicalExpression>> exprs, String functionName)
+ throws AlgebricksException {
+ // Check the first parameter - Expression1. If it's a constant, then we can check the type here.
+ ILogicalExpression firstExpr = exprs.get(0).getValue();
+ if (firstExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
+ && ConstantExpressionUtil.getConstantIaObjectType(firstExpr) != ATypeTag.STRING) {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, firstExpr.getSourceLocation(),
+ functionName, ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
+ }
+
+ // Check the second parameter - Expression2. If it's a constant, then we can check the type here.
+ ILogicalExpression secondExpr = exprs.get(1).getValue();
+ if (secondExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ ATypeTag exprTypeTag = ConstantExpressionUtil.getConstantIaObjectType(secondExpr);
+ switch (exprTypeTag) {
+ case STRING:
+ case MULTISET:
+ case ARRAY:
+ break;
+ default:
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, secondExpr.getSourceLocation(),
+ functionName, exprTypeTag);
+ }
+ }
+ }
+
+ /**
+ * Checks the option of the given ftcontains() function. Also, sets default value.
+ *
+ * @param expr
+ * @throws AlgebricksException
+ *
+ * @return the full-text config name if specified in the function option,
+ * null if not specified which implies the default full-text config will be utilized later
+ */
+ private String checkValueForThirdParameterAndGetFullTextConfig(Mutable<ILogicalExpression> expr,
+ List<Mutable<ILogicalExpression>> newArgs, String functionName) throws AlgebricksException {
+ String ftConfigName = null;
+
+ // Get the last parameter - this should be a record-constructor.
+ AbstractFunctionCallExpression openRecConsExpr = (AbstractFunctionCallExpression) expr.getValue();
+ FunctionIdentifier openRecConsFi = openRecConsExpr.getFunctionIdentifier();
+ if (openRecConsFi != BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR
+ && openRecConsFi != BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR) {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, openRecConsExpr.getSourceLocation(),
+ functionName, openRecConsFi);
+ }
+
+ // We multiply 2 because the layout of the arguments are: [expr, val, expr1, val1, ...]
+ if (openRecConsExpr.getArguments().size() > FullTextContainsFunctionDescriptor.getParamTypeMap().size()
+ * 2) {
+ throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION,
+ openRecConsExpr.getSourceLocation(), functionName);
+ }
+
+ if (openRecConsExpr.getArguments().size() % 2 != 0) {
+ throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
+ openRecConsExpr.getSourceLocation(), functionName);
+ }
+
+ for (int i = 0; i < openRecConsExpr.getArguments().size(); i = i + 2) {
+ ILogicalExpression optionExpr = openRecConsExpr.getArguments().get(i).getValue();
+ ILogicalExpression optionExprVal = openRecConsExpr.getArguments().get(i + 1).getValue();
+
+ String option = ConstantExpressionUtil.getStringConstant(optionExpr);
+
+ if (optionExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT || option == null) {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExpr.getSourceLocation(),
+ functionName, optionExpr.getExpressionTag());
+ }
+
+ option = option.toLowerCase();
+ if (!FullTextContainsFunctionDescriptor.getParamTypeMap().containsKey(option)) {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExprVal.getSourceLocation(),
+ functionName, option);
+ }
+
+ String optionTypeStringVal = null;
+ // If the option value is a constant, then we can check here.
+ if (optionExprVal.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ switch (FullTextContainsFunctionDescriptor.getParamTypeMap().get(option)) {
+ case STRING:
+ optionTypeStringVal = ConstantExpressionUtil.getStringConstant(optionExprVal);
+ if (optionTypeStringVal == null) {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+ optionExprVal.getSourceLocation(), functionName, option);
+ }
+ optionTypeStringVal = optionTypeStringVal.toLowerCase();
+ break;
+ default:
+ // Currently, we only have a string parameter. So, the flow doesn't reach here.
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+ optionExprVal.getSourceLocation(), functionName, option);
+ }
+
+ // Check the validity of option value
+ switch (option) {
+ case FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION:
+ checkSearchModeOption(optionTypeStringVal, functionName, optionExprVal.getSourceLocation());
+ break;
+ case FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION:
+ checkFullTextConfigOption(optionTypeStringVal, functionName,
+ optionExprVal.getSourceLocation());
+ ftConfigName = optionTypeStringVal;
+ break;
+ default:
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
+ optionExprVal.getSourceLocation(), functionName, option);
+ }
+ }
+
+ // Add this option as arguments to the ftcontains().
+ newArgs.add(new MutableObject<ILogicalExpression>(optionExpr));
+ newArgs.add(new MutableObject<ILogicalExpression>(optionExprVal));
+ }
+ return ftConfigName;
+ }
+
+ private void checkSearchModeOption(String optionVal, String functionName, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ if (optionVal.equals(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())
+ || optionVal.equals(FullTextContainsFunctionDescriptor.SearchMode.ANY.getValue())) {
+ return;
+ } else {
+ throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, functionName, optionVal);
+ }
+ }
+
+ private void checkFullTextConfigOption(String optionVal, String functionName, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ // Currently, here we only check if the full-text config is null or empty string
+ // We will check if the full-text config exists at run time
+ if (Strings.isNullOrEmpty(optionVal)) {
+ throw CompilationException.create(ErrorCode.COMPILATION_INVALID_EXPRESSION, sourceLoc, functionName,
+ FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION, "not-null", "null");
+ } else {
+ return;
+ }
+ }
+
+ /**
+ * Sets the default option value(s) when a user doesn't provide any option.
+ */
+ void setDefaultValueForThirdParameter(List<Mutable<ILogicalExpression>> newArgs) {
+ // Sets the search mode option: the default option is conjunctive search.
+ ILogicalExpression searchModeOptionExpr = new ConstantExpression(
+ new AsterixConstantValue(new AString(FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION)));
+ ILogicalExpression searchModeValExpr = new ConstantExpression(new AsterixConstantValue(
+ new AString(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())));
+ // Add this option as arguments to the ftcontains().
+ newArgs.add(new MutableObject<ILogicalExpression>(searchModeOptionExpr));
+ newArgs.add(new MutableObject<ILogicalExpression>(searchModeValExpr));
+
+ // We don't set the full-text config option here because the default value should be null
+ }
+
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
deleted file mode 100644
index c565b34..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FullTextContainsParameterCheckRule.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.
- */
-package org.apache.asterix.optimizer.rules;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.constants.AsterixConstantValue;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
-import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-
-/**
- * Checks whether the given parameters of the ftcontains() function are correct during the compilation.
- */
-public class FullTextContainsParameterCheckRule implements IAlgebraicRewriteRule {
-
- // Visitor for checking and transforming ftcontains() expression
- protected FullTextContainsExpressionVisitor ftcontainsExprVisitor = new FullTextContainsExpressionVisitor();
-
- @Override
- public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- if (context.checkIfInDontApplySet(this, opRef.getValue())) {
- return false;
- }
- if (checkParameter(opRef, context)) {
- OperatorPropertiesUtil.typeOpRec(opRef, context);
- return true;
- }
- return false;
- }
-
- /**
- * Check the correctness of the parameters of the ftcontains(). Also rearrange options as arguments.
- * The expected form of ftcontains() is ftcontains(expression1, expression2, parameters as a record).
- */
- private boolean checkParameter(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
- AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
- boolean modified = op.acceptExpressionTransform(ftcontainsExprVisitor);
- if (modified) {
- context.addToDontApplySet(this, op);
- }
- return modified;
- }
-
- /**
- * This visitor class handles actual checking and transformation.
- */
- protected class FullTextContainsExpressionVisitor implements ILogicalExpressionReferenceTransform {
-
- // the last expression position before the option argument in the arguments array
- private static final int LAST_EXPRESSION_POS_BEFORE_OPTION = 1;
- // The number of anticipated arguments for a full-text query when a user doesn't provide any option.
- private static final int FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS = 2;
- // The number of anticipated arguments for a full-text query when a user provide option(s) as a record.
- private static final int FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS = 3;
-
- public FullTextContainsExpressionVisitor() {
- // no parameter is needed.
- }
-
- @Override
- public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
- ILogicalExpression e = exprRef.getValue();
- switch (e.getExpressionTag()) {
- case FUNCTION_CALL:
- return transformFunctionCallExpression((AbstractFunctionCallExpression) e);
- default:
- return false;
- }
- }
-
- private boolean transformFunctionCallExpression(AbstractFunctionCallExpression fce) throws AlgebricksException {
- boolean modified = false;
- FunctionIdentifier fi = fce.getFunctionIdentifier();
- if (fi != BuiltinFunctions.FULLTEXT_CONTAINS && fi != BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
- for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
- if (transform(arg)) {
- modified = true;
- }
- }
- } else {
- modified = checkParameterForFuncExpr(fce, fi);
- }
-
- return modified;
- }
-
- private boolean checkParameterForFuncExpr(AbstractFunctionCallExpression funcExpr, FunctionIdentifier fi)
- throws AlgebricksException {
- // Collects the correct number of arguments - it can be 2 if a user doesn't provide any option.
- int numberOfCorrectArguments = 0;
- String functionName = "";
- if (fi == BuiltinFunctions.FULLTEXT_CONTAINS) {
- numberOfCorrectArguments = FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS;
- functionName = BuiltinFunctions.FULLTEXT_CONTAINS.getName();
- } else if (fi == BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION) {
- numberOfCorrectArguments = FULLTEXT_QUERY_WITHOUT_OPTION_NO_OF_ARGUMENTS;
- functionName = BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION.getName();
- }
-
- // If numberOfCorrectArguments is greater than zero, then this is a full-text search query.
- if (numberOfCorrectArguments > 0) {
-
- List<Mutable<ILogicalExpression>> oldExprs = funcExpr.getArguments();
- List<Mutable<ILogicalExpression>> newExprs = new ArrayList<>();
-
- // The number of parameters should be three: exp1, exp2, and the option
- if (oldExprs.size() != numberOfCorrectArguments) {
- throw CompilationException.create(ErrorCode.COMPILATION_INVALID_PARAMETER_NUMBER,
- funcExpr.getSourceLocation(), fi.getName(), oldExprs.size());
- }
-
- // The last expression before the option needs to be copied first.
- for (int i = 0; i <= LAST_EXPRESSION_POS_BEFORE_OPTION; i++) {
- newExprs.add(new MutableObject<ILogicalExpression>(oldExprs.get(i).getValue()));
- }
-
- // Sanity check for the types of the first two parameters
- checkFirstAndSecondParamter(oldExprs, functionName);
-
- // Checks and transforms the actual full-text parameters.
- if (numberOfCorrectArguments == FULLTEXT_QUERY_WITH_OPTION_NO_OF_ARGUMENTS) {
- checkValueForThirdParameter(oldExprs.get(2), newExprs, functionName);
- } else {
- // no option provided case: sets the default option here.
- setDefaultValueForThirdParameter(newExprs);
- }
-
- // Resets the last argument.
- funcExpr.getArguments().clear();
- funcExpr.getArguments().addAll(newExprs);
-
- return true;
- }
-
- return false;
- }
-
- /**
- * Checks the correctness of the first and second argument. If the argument is a constant, we can check
- * it now. If the argument is not a constant, we will defer the checking until run-time.
- */
- private void checkFirstAndSecondParamter(List<Mutable<ILogicalExpression>> exprs, String functionName)
- throws AlgebricksException {
- // Check the first parameter - Expression1. If it's a constant, then we can check the type here.
- ILogicalExpression firstExpr = exprs.get(0).getValue();
- if (firstExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT
- && ConstantExpressionUtil.getConstantIaObjectType(firstExpr) != ATypeTag.STRING) {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, firstExpr.getSourceLocation(),
- functionName, ConstantExpressionUtil.getConstantIaObjectType(firstExpr));
- }
-
- // Check the second parameter - Expression2. If it's a constant, then we can check the type here.
- ILogicalExpression secondExpr = exprs.get(1).getValue();
- if (secondExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
- ATypeTag exprTypeTag = ConstantExpressionUtil.getConstantIaObjectType(secondExpr);
- switch (exprTypeTag) {
- case STRING:
- case MULTISET:
- case ARRAY:
- break;
- default:
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, secondExpr.getSourceLocation(),
- functionName, exprTypeTag);
- }
- }
- }
-
- /**
- * Checks the option of the given ftcontains() function. Also, sets default value.
- *
- * @param expr
- * @throws AlgebricksException
- */
- private void checkValueForThirdParameter(Mutable<ILogicalExpression> expr,
- List<Mutable<ILogicalExpression>> newArgs, String functionName) throws AlgebricksException {
- // Get the last parameter - this should be a record-constructor.
- AbstractFunctionCallExpression openRecConsExpr = (AbstractFunctionCallExpression) expr.getValue();
- FunctionIdentifier openRecConsFi = openRecConsExpr.getFunctionIdentifier();
- if (openRecConsFi != BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR
- && openRecConsFi != BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR) {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, openRecConsExpr.getSourceLocation(),
- functionName, openRecConsFi);
- }
-
- // We multiply 2 because the layout of the arguments are: [expr, val, expr1, val1, ...]
- if (openRecConsExpr.getArguments().size() > FullTextContainsDescriptor.getParamTypeMap().size() * 2) {
- throw CompilationException.create(ErrorCode.TOO_MANY_OPTIONS_FOR_FUNCTION,
- openRecConsExpr.getSourceLocation(), functionName);
- }
-
- for (int i = 0; i < openRecConsExpr.getArguments().size(); i = i + 2) {
- ILogicalExpression optionExpr = openRecConsExpr.getArguments().get(i).getValue();
- ILogicalExpression optionExprVal = openRecConsExpr.getArguments().get(i + 1).getValue();
-
- String option = ConstantExpressionUtil.getStringConstant(optionExpr);
-
- if (optionExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT || option == null) {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExpr.getSourceLocation(),
- functionName, optionExpr.getExpressionTag());
- }
-
- option = option.toLowerCase();
- if (!FullTextContainsDescriptor.getParamTypeMap().containsKey(option)) {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, optionExprVal.getSourceLocation(),
- functionName, option);
- }
-
- String optionTypeStringVal = null;
-
- // If the option value is a constant, then we can check here.
- if (optionExprVal.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
- switch (FullTextContainsDescriptor.getParamTypeMap().get(option)) {
- case STRING:
- optionTypeStringVal = ConstantExpressionUtil.getStringConstant(optionExprVal);
- if (optionTypeStringVal == null) {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
- optionExprVal.getSourceLocation(), functionName, option);
- }
- optionTypeStringVal = optionTypeStringVal.toLowerCase();
- break;
- default:
- // Currently, we only have a string parameter. So, the flow doesn't reach here.
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
- optionExprVal.getSourceLocation(), functionName, option);
- }
-
- // Check the validity of option value
- switch (option) {
- case FullTextContainsDescriptor.SEARCH_MODE_OPTION:
- checkSearchModeOption(optionTypeStringVal, functionName, optionExprVal.getSourceLocation());
- break;
- default:
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED,
- optionExprVal.getSourceLocation(), functionName, option);
- }
- }
-
- // Add this option as arguments to the ftcontains().
- newArgs.add(new MutableObject<ILogicalExpression>(optionExpr));
- newArgs.add(new MutableObject<ILogicalExpression>(optionExprVal));
- }
- }
-
- private void checkSearchModeOption(String optionVal, String functionName, SourceLocation sourceLoc)
- throws AlgebricksException {
- if (optionVal.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)
- || optionVal.equals(FullTextContainsDescriptor.DISJUNCTIVE_SEARCH_MODE_OPTION)) {
- return;
- } else {
- throw CompilationException.create(ErrorCode.TYPE_UNSUPPORTED, sourceLoc, functionName, optionVal);
- }
- }
-
- /**
- * Sets the default option value(s) when a user doesn't provide any option.
- */
- void setDefaultValueForThirdParameter(List<Mutable<ILogicalExpression>> newArgs) throws AlgebricksException {
- // Sets the search mode option: the default option is conjunctive search.
- ILogicalExpression searchModeOptionExpr = new ConstantExpression(
- new AsterixConstantValue(new AString(FullTextContainsDescriptor.SEARCH_MODE_OPTION)));
- ILogicalExpression searchModeValExpr = new ConstantExpression(
- new AsterixConstantValue(new AString(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)));
-
- // Add this option as arguments to the ftcontains().
- newArgs.add(new MutableObject<ILogicalExpression>(searchModeOptionExpr));
- newArgs.add(new MutableObject<ILogicalExpression>(searchModeValExpr));
- }
-
- }
-
-}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 07247fb..c3859be 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -18,11 +18,15 @@
*/
package org.apache.asterix.optimizer.rules;
+import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Deque;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.stream.Collectors;
import org.apache.asterix.algebra.operators.CommitOperator;
@@ -38,6 +42,7 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.AString;
@@ -57,6 +62,7 @@
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
@@ -65,17 +71,24 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -87,6 +100,8 @@
* assign --> insert-delete-upsert --> *(secondary indexes index-insert-delete-upsert) --> sink
*/
public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewriteRule {
+ private IOptimizationContext context;
+ private SourceLocation sourceLoc;
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -121,7 +136,8 @@
primaryIndexModificationOp.getAdditionalNonFilteringExpressions();
LogicalVariable newRecordVar;
LogicalVariable newMetaVar = null;
- SourceLocation sourceLoc = primaryIndexModificationOp.getSourceLocation();
+ sourceLoc = primaryIndexModificationOp.getSourceLocation();
+ this.context = context;
/**
* inputOp is the assign operator which extracts primary keys from the input
@@ -129,14 +145,14 @@
*/
AbstractLogicalOperator inputOp =
(AbstractLogicalOperator) primaryIndexModificationOp.getInputs().get(0).getValue();
- newRecordVar = getRecordVar(context, inputOp, newRecordExpr, 0);
+ newRecordVar = getRecordVar(inputOp, newRecordExpr, 0);
if (newMetaExprs != null && !newMetaExprs.isEmpty()) {
if (newMetaExprs.size() > 1) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
"Number of meta records can't be more than 1. Number of meta records found = "
+ newMetaExprs.size());
}
- newMetaVar = getRecordVar(context, inputOp, newMetaExprs.get(0).getValue(), 1);
+ newMetaVar = getRecordVar(inputOp, newMetaExprs.get(0).getValue(), 1);
}
/*
@@ -177,9 +193,9 @@
// for insert, primary key index is handled together when primary index
indexes = indexes.stream().filter(index -> !index.isPrimaryKeyIndex()).collect(Collectors.toList());
}
+
// Set the top operator pointer to the primary IndexInsertDeleteOperator
ILogicalOperator currentTop = primaryIndexModificationOp;
- boolean hasSecondaryIndex = false;
// Put an n-gram or a keyword index in the later stage of index-update,
// since TokenizeOperator needs to be involved.
@@ -216,7 +232,7 @@
// Replicate Operator is applied only when doing the bulk-load.
ReplicateOperator replicateOp = null;
- if (secondaryIndexTotalCnt > 1 && primaryIndexModificationOp.isBulkload()) {
+ if (secondaryIndexTotalCnt > 1 && isBulkload) {
// Split the logical plan into "each secondary index update branch"
// to replicate each <PK,OBJECT> pair.
replicateOp = new ReplicateOperator(secondaryIndexTotalCnt);
@@ -247,8 +263,8 @@
* is solved
*/
|| primaryIndexModificationOp.getOperation() == Kind.DELETE) {
- injectFieldAccessesForIndexes(context, dataset, indexes, fieldVarsForNewRecord, recType, metaType,
- newRecordVar, newMetaVar, primaryIndexModificationOp, false);
+ injectFieldAccessesForIndexes(dataset, indexes, fieldVarsForNewRecord, recType, metaType, newRecordVar,
+ newMetaVar, primaryIndexModificationOp, false);
if (replicateOp != null) {
context.computeAndSetTypeEnvironmentForOperator(replicateOp);
}
@@ -260,38 +276,61 @@
*/) {
List<LogicalVariable> beforeOpMetaVars = primaryIndexModificationOp.getBeforeOpAdditionalNonFilteringVars();
LogicalVariable beforeOpMetaVar = beforeOpMetaVars == null ? null : beforeOpMetaVars.get(0);
- currentTop = injectFieldAccessesForIndexes(context, dataset, indexes, fieldVarsForBeforeOperation, recType,
- metaType, primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, currentTop, true);
+ currentTop = injectFieldAccessesForIndexes(dataset, indexes, fieldVarsForBeforeOperation, recType, metaType,
+ primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, currentTop, true);
}
- // Iterate each secondary index and applying Index Update operations.
- // At first, op1 is the index insert op insertOp
+ // Add the appropriate SIDX maintenance operations.
for (Index index : indexes) {
if (!index.isSecondaryIndex()) {
continue;
}
- hasSecondaryIndex = true;
+
// Get the secondary fields names and types
- List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
- List<IAType> secondaryKeyTypes = index.getKeyFieldTypes();
+ List<List<String>> secondaryKeyFields = null;
+ List<IAType> secondaryKeyTypes = null;
+ List<Integer> secondaryKeySources = null;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ secondaryKeyFields = valueIndexDetails.getKeyFieldNames();
+ secondaryKeyTypes = valueIndexDetails.getKeyFieldTypes();
+ secondaryKeySources = valueIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ secondaryKeyFields = textIndexDetails.getKeyFieldNames();
+ secondaryKeyTypes = textIndexDetails.getKeyFieldTypes();
+ secondaryKeySources = textIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ case ARRAY:
+ // These details are handled separately for array indexes.
+ break;
+ default:
+ continue;
+ }
+
+ // Set our key variables and expressions for non-array indexes. Our secondary keys for array indexes will
+ // always be an empty list.
List<LogicalVariable> secondaryKeyVars = new ArrayList<>();
List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<>();
List<Mutable<ILogicalExpression>> beforeOpSecondaryExpressions = new ArrayList<>();
ILogicalOperator replicateOutput;
-
- for (int i = 0; i < secondaryKeyFields.size(); i++) {
- IndexFieldId indexFieldId = new IndexFieldId(index.getKeyFieldSourceIndicators().get(i),
- secondaryKeyFields.get(i), secondaryKeyTypes.get(i).getTypeTag());
- LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
- secondaryKeyVars.add(skVar);
- VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
- skVarRef.setSourceLocation(sourceLoc);
- secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
- if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
- VariableReferenceExpression varRef =
- new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
- varRef.setSourceLocation(sourceLoc);
- beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
+ if (!index.getIndexType().equals(IndexType.ARRAY)) {
+ for (int i = 0; i < secondaryKeyFields.size(); i++) {
+ IndexFieldId indexFieldId = new IndexFieldId(secondaryKeySources.get(i), secondaryKeyFields.get(i),
+ secondaryKeyTypes.get(i).getTypeTag());
+ LogicalVariable skVar = fieldVarsForNewRecord.get(indexFieldId);
+ secondaryKeyVars.add(skVar);
+ VariableReferenceExpression skVarRef = new VariableReferenceExpression(skVar);
+ skVarRef.setSourceLocation(sourceLoc);
+ secondaryExpressions.add(new MutableObject<ILogicalExpression>(skVarRef));
+ if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+ VariableReferenceExpression varRef =
+ new VariableReferenceExpression(fieldVarsForBeforeOperation.get(indexFieldId));
+ varRef.setSourceLocation(sourceLoc);
+ beforeOpSecondaryExpressions.add(new MutableObject<ILogicalExpression>(varRef));
+ }
}
}
@@ -301,12 +340,13 @@
Mutable<ILogicalExpression> filterExpression =
(primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
: createFilterExpression(secondaryKeyVars, context.getOutputTypeEnvironment(currentTop),
- index.isOverridingKeyFieldTypes(), sourceLoc);
+ index.getIndexDetails().isOverridingKeyFieldTypes());
DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
// Introduce the TokenizeOperator only when doing bulk-load,
// and index type is keyword or n-gram.
- if (index.getIndexType() != IndexType.BTREE && primaryIndexModificationOp.isBulkload()) {
+ if (index.getIndexType() != IndexType.BTREE && index.getIndexType() != IndexType.ARRAY
+ && primaryIndexModificationOp.isBulkload()) {
// Note: Bulk load case, we don't need to take care of it for upsert operation
// Check whether the index is length-partitioned or not.
// If partitioned, [input variables to TokenizeOperator,
@@ -330,8 +370,8 @@
// Check the field type of the secondary key.
IAType secondaryKeyType;
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
- index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+ secondaryKeyFields.get(0), recType);
secondaryKeyType = keyPairType.first;
List<Object> varTypes = new ArrayList<>();
@@ -399,11 +439,82 @@
}
}
indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
+
+ // For array indexes we have no secondary keys to reference. We must add separate branches to
+ // first extract our keys.
+ if (index.getIndexType() == IndexType.ARRAY && !isBulkload) {
+ NestedTupleSourceOperator unnestSourceOp =
+ new NestedTupleSourceOperator(new MutableObject<>(indexUpdate));
+ unnestSourceOp.setSourceLocation(sourceLoc);
+ context.computeAndSetTypeEnvironmentForOperator(unnestSourceOp);
+ UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(unnestSourceOp, index, newRecordVar,
+ newMetaVar, recType, metaType, dataset.hasMetaPart());
+ unnestSIDXBranch.applyProjectDistinct();
+
+ // If there exists a filter expression, add it to the top of our nested plan.
+ filterExpression = (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
+ : createFilterExpression(unnestSIDXBranch.lastFieldVars,
+ context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
+ index.getIndexDetails().isOverridingKeyFieldTypes());
+ if (filterExpression != null) {
+ unnestSIDXBranch.applyFilteringExpression(filterExpression);
+ }
+
+ // Finalize our nested plan.
+ ILogicalPlan unnestPlan = unnestSIDXBranch.buildBranch();
+ indexUpdate.getNestedPlans().add(unnestPlan);
+
+ // If we have an UPSERT, then create and add a branch to extract our old keys as well.
+ if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
+ NestedTupleSourceOperator unnestBeforeSourceOp =
+ new NestedTupleSourceOperator(new MutableObject<>(indexUpdate));
+ unnestBeforeSourceOp.setSourceLocation(sourceLoc);
+ context.computeAndSetTypeEnvironmentForOperator(unnestBeforeSourceOp);
+
+ List<LogicalVariable> beforeOpMetaVars =
+ primaryIndexModificationOp.getBeforeOpAdditionalNonFilteringVars();
+ LogicalVariable beforeOpMetaVar = beforeOpMetaVars == null ? null : beforeOpMetaVars.get(0);
+ UnnestBranchCreator unnestBeforeSIDXBranch = buildUnnestBranch(unnestBeforeSourceOp, index,
+ primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, recType,
+ metaType, dataset.hasMetaPart());
+ unnestBeforeSIDXBranch.applyProjectDistinct();
+ indexUpdate.getNestedPlans().add(unnestBeforeSIDXBranch.buildBranch());
+ }
+ } else if (index.getIndexType() == IndexType.ARRAY && isBulkload) {
+ // If we have a bulk load, we must sort the entire input by <SK, PK>. Do not use any
+ // nested plans here.
+ UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(currentTop, index, newRecordVar,
+ newMetaVar, recType, metaType, dataset.hasMetaPart());
+ unnestSIDXBranch.applyProjectDistinct(primaryIndexModificationOp.getPrimaryKeyExpressions(),
+ primaryIndexModificationOp.getAdditionalFilteringExpressions());
+ indexUpdate.getInputs().clear();
+ introduceNewOp(unnestSIDXBranch.currentTop, indexUpdate, true);
+
+ // Update the secondary expressions of our index.
+ secondaryExpressions = new ArrayList<>();
+ for (LogicalVariable var : unnestSIDXBranch.lastFieldVars) {
+ secondaryExpressions.add(new MutableObject<>(new VariableReferenceExpression(var)));
+ }
+ indexUpdate.setSecondaryKeyExprs(secondaryExpressions);
+
+ // Update the filter expression to include these new keys.
+ filterExpression = createFilterExpression(unnestSIDXBranch.lastFieldVars,
+ context.getOutputTypeEnvironment(unnestSIDXBranch.currentTop),
+ index.getIndexDetails().isOverridingKeyFieldTypes());
+ indexUpdate.setFilterExpression(filterExpression);
+
+ if (replicateOp != null) {
+ // If we have a replicate, then update the replicate operator to include this branch.
+ replicateOp.getOutputs().add(new MutableObject<>(unnestSIDXBranch.currentBottom));
+ op0.getInputs().add(new MutableObject<ILogicalOperator>(indexUpdate));
+ continue;
+ }
+ }
}
} else {
// Get type, dimensions and number of keys
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- secondaryKeyFields.get(0), recType);
+ Pair<IAType, Boolean> keyPairType =
+ Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
IAType spatialType = keyPairType.first;
boolean isPointMBR =
spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -486,7 +597,7 @@
// nullable.
boolean forceFilter = keyPairType.second;
filterExpression = createFilterExpression(keyVarList,
- context.getOutputTypeEnvironment(assignCoordinates), forceFilter, sourceLoc);
+ context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
}
DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
@@ -533,7 +644,7 @@
* "blocking" sort operator since tuples are already sorted. We mark the materialization flag for that
* branch to make it blocking. Without "blocking", the activity cluster graph would be messed up
*/
- if (index.getKeyFieldNames().isEmpty() && index.getIndexType() == IndexType.BTREE) {
+ if (index.isPrimaryKeyIndex()) {
int positionOfSecondaryPrimaryIndex = replicateOp.getOutputs().size() - 1;
replicateOp.getOutputMaterializationFlags()[positionOfSecondaryPrimaryIndex] = true;
}
@@ -542,10 +653,6 @@
// For bulk load, we connect all fanned out insert operator to a single SINK operator
op0.getInputs().add(new MutableObject<ILogicalOperator>(indexUpdate));
}
-
- }
- if (!hasSecondaryIndex) {
- return false;
}
if (!primaryIndexModificationOp.isBulkload()) {
@@ -558,8 +665,101 @@
return true;
}
- private LogicalVariable getRecordVar(IOptimizationContext context, AbstractLogicalOperator inputOp,
- ILogicalExpression recordExpr, int expectedRecordIndex) throws AlgebricksException {
+ private UnnestBranchCreator buildUnnestBranch(ILogicalOperator unnestSourceOp, Index index,
+ LogicalVariable recordVar, LogicalVariable metaVar, ARecordType recType, ARecordType metaType,
+ boolean hasMetaPart) throws AlgebricksException {
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+
+ // First, locate a field having the required UNNEST path. Queue this first, and all other keys will follow.
+ Deque<Integer> keyPositionQueue = new ArrayDeque<>();
+ for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
+ Index.ArrayIndexElement e = arrayIndexDetails.getElementList().get(i);
+ if (e.getUnnestList().isEmpty()) {
+ keyPositionQueue.addLast(i);
+ } else {
+ keyPositionQueue.addFirst(i);
+ }
+ }
+
+ // Get the record variable associated with our record path.
+ Index.ArrayIndexElement workingElement = arrayIndexDetails.getElementList().get(keyPositionQueue.getFirst());
+ int sourceIndicatorForBaseRecord = workingElement.getSourceIndicator();
+ LogicalVariable sourceVarForBaseRecord = hasMetaPart
+ ? ((sourceIndicatorForBaseRecord == Index.RECORD_INDICATOR) ? recordVar : metaVar) : recordVar;
+ UnnestBranchCreator branchCreator = new UnnestBranchCreator(sourceVarForBaseRecord, unnestSourceOp);
+
+ int initialKeyPositionQueueSize = keyPositionQueue.size();
+ Set<LogicalVariable> secondaryKeyVars = new HashSet<>();
+ for (int i = 0; i < initialKeyPositionQueueSize; i++) {
+
+ // Poll from our queue, and get a key position.
+ int workingKeyPos = keyPositionQueue.pollFirst();
+ workingElement = arrayIndexDetails.getElementList().get(workingKeyPos);
+ int sourceIndicator = workingElement.getSourceIndicator();
+ ARecordType recordType =
+ hasMetaPart ? ((sourceIndicator == Index.RECORD_INDICATOR) ? recType : metaType) : recType;
+
+ boolean isOpenOrNestedField;
+ if (workingElement.getUnnestList().isEmpty()) {
+ // We have an atomic element (i.e. we have a composite array index).
+ List<String> atomicFieldName = workingElement.getProjectList().get(0);
+ isOpenOrNestedField =
+ (atomicFieldName.size() != 1) || !recordType.isClosedField(atomicFieldName.get(0));
+
+ // The UNNEST path has already been created (we queued this first), so we look at the current top.
+ LogicalVariable newVar = context.newVar();
+ VariableReferenceExpression varRef = new VariableReferenceExpression(sourceVarForBaseRecord);
+ varRef.setSourceLocation(sourceLoc);
+ AbstractFunctionCallExpression newVarRef = (isOpenOrNestedField)
+ ? getFieldAccessFunction(new MutableObject<>(varRef),
+ recordType.getFieldIndex(atomicFieldName.get(0)), atomicFieldName)
+ : getFieldAccessFunction(new MutableObject<>(varRef), -1, atomicFieldName);
+
+ AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
+ newAssignOp.setSourceLocation(sourceLoc);
+ branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
+ secondaryKeyVars.add(newVar);
+
+ } else {
+ // We have an array element. The "open / nestedness" is determined by the first UNNEST field.
+ isOpenOrNestedField = workingElement.getUnnestList().get(0).size() > 1
+ || !recordType.isClosedField(workingElement.getUnnestList().get(0).get(0));
+
+ // Walk the array path.
+ List<String> flatFirstFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(
+ workingElement.getUnnestList(), workingElement.getProjectList().get(0));
+ List<Integer> firstArrayIndicator = ArrayIndexUtil
+ .getArrayDepthIndicator(workingElement.getUnnestList(), workingElement.getProjectList().get(0));
+ ArrayIndexUtil.walkArrayPath((isOpenOrNestedField) ? null : recordType, flatFirstFieldName,
+ firstArrayIndicator, branchCreator);
+
+ // For all other elements in the PROJECT list, add an assign.
+ for (int j = 1; j < workingElement.getProjectList().size(); j++) {
+ LogicalVariable newVar = context.newVar();
+ AbstractFunctionCallExpression newVarRef =
+ getFieldAccessFunction(new MutableObject<>(branchCreator.createLastRecordVarRef()), -1,
+ workingElement.getProjectList().get(j));
+
+ AssignOperator newAssignOp = new AssignOperator(newVar, new MutableObject<>(newVarRef));
+ newAssignOp.setSourceLocation(sourceLoc);
+ branchCreator.currentTop = introduceNewOp(branchCreator.currentTop, newAssignOp, true);
+ secondaryKeyVars.add(newVar);
+ }
+ }
+
+ branchCreator.lowerIsFirstWalkFlag();
+ secondaryKeyVars.addAll(branchCreator.lastFieldVars);
+ }
+
+ // Update the variables we are to use for the head operators.
+ branchCreator.lastFieldVars.clear();
+ branchCreator.lastFieldVars.addAll(secondaryKeyVars);
+
+ return branchCreator;
+ }
+
+ private LogicalVariable getRecordVar(AbstractLogicalOperator inputOp, ILogicalExpression recordExpr,
+ int expectedRecordIndex) throws AlgebricksException {
if (exprIsRecord(context.getOutputTypeEnvironment(inputOp), recordExpr)) {
return ((VariableReferenceExpression) recordExpr).getVariableReference();
} else {
@@ -599,21 +799,39 @@
return false;
}
- private ILogicalOperator injectFieldAccessesForIndexes(IOptimizationContext context, Dataset dataset,
- List<Index> indexes, Map<IndexFieldId, LogicalVariable> fieldAccessVars, ARecordType recType,
- ARecordType metaType, LogicalVariable recordVar, LogicalVariable metaVar, ILogicalOperator currentTop,
- boolean afterOp) throws AlgebricksException {
+ private ILogicalOperator injectFieldAccessesForIndexes(Dataset dataset, List<Index> indexes,
+ Map<IndexFieldId, LogicalVariable> fieldAccessVars, ARecordType recType, ARecordType metaType,
+ LogicalVariable recordVar, LogicalVariable metaVar, ILogicalOperator currentTop, boolean afterOp)
+ throws AlgebricksException {
List<LogicalVariable> vars = new ArrayList<>();
List<Mutable<ILogicalExpression>> exprs = new ArrayList<>();
SourceLocation sourceLoc = currentTop.getSourceLocation();
for (Index index : indexes) {
- if (index.isPrimaryIndex()) {
+ if (index.isPrimaryIndex() || index.getIndexType() == IndexType.ARRAY) {
+ // Array indexes require UNNESTs, which must be handled after the PIDX op.
continue;
}
- List<IAType> skTypes = index.getKeyFieldTypes();
- List<List<String>> skNames = index.getKeyFieldNames();
- List<Integer> indicators = index.getKeyFieldSourceIndicators();
- for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+ List<List<String>> skNames;
+ List<IAType> skTypes;
+ List<Integer> indicators;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ skNames = valueIndexDetails.getKeyFieldNames();
+ skTypes = valueIndexDetails.getKeyFieldTypes();
+ indicators = valueIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ skNames = textIndexDetails.getKeyFieldNames();
+ skTypes = textIndexDetails.getKeyFieldTypes();
+ indicators = textIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ String.valueOf(index.getIndexType()));
+ }
+ for (int i = 0; i < skNames.size(); i++) {
IndexFieldId indexFieldId =
new IndexFieldId(indicators.get(i), skNames.get(i), skTypes.get(i).getTypeTag());
if (fieldAccessVars.containsKey(indexFieldId)) {
@@ -635,8 +853,8 @@
// make handling of records with incorrect value type for this field easier and cleaner
context.addNotToBeInlinedVar(fieldVar);
// create field access
- AbstractFunctionCallExpression fieldAccessFunc = getOpenOrNestedFieldAccessFunction(
- new MutableObject<>(varRef), indexFieldId.fieldName, sourceLoc);
+ AbstractFunctionCallExpression fieldAccessFunc =
+ getFieldAccessFunction(new MutableObject<>(varRef), -1, indexFieldId.fieldName);
// create cast
theFieldAccessFunc = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(
index.isEnforced() ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX));
@@ -649,24 +867,25 @@
int pos = indexFieldId.fieldName.size() > 1 ? -1
: sourceType.getFieldIndex(indexFieldId.fieldName.get(0));
// Field not found --> This is either an open field or a nested field. it can't be accessed by index
- theFieldAccessFunc = (pos == -1)
- ? getOpenOrNestedFieldAccessFunction(new MutableObject<>(varRef), indexFieldId.fieldName,
- sourceLoc)
- : getClosedFieldAccessFunction(new MutableObject<>(varRef), pos, sourceLoc);
+ theFieldAccessFunc =
+ getFieldAccessFunction(new MutableObject<>(varRef), pos, indexFieldId.fieldName);
}
vars.add(fieldVar);
exprs.add(new MutableObject<ILogicalExpression>(theFieldAccessFunc));
fieldAccessVars.put(indexFieldId, fieldVar);
}
}
- // AssignOperator assigns secondary keys to their vars
- AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
- castedFieldAssignOperator.setSourceLocation(sourceLoc);
- return introduceNewOp(context, currentTop, castedFieldAssignOperator, afterOp);
+ if (!vars.isEmpty()) {
+ // AssignOperator assigns secondary keys to their vars
+ AssignOperator castedFieldAssignOperator = new AssignOperator(vars, exprs);
+ castedFieldAssignOperator.setSourceLocation(sourceLoc);
+ return introduceNewOp(currentTop, castedFieldAssignOperator, afterOp);
+ }
+ return currentTop;
}
- private static ILogicalOperator introduceNewOp(IOptimizationContext context, ILogicalOperator currentTopOp,
- ILogicalOperator newOp, boolean afterOp) throws AlgebricksException {
+ private ILogicalOperator introduceNewOp(ILogicalOperator currentTopOp, ILogicalOperator newOp, boolean afterOp)
+ throws AlgebricksException {
if (afterOp) {
newOp.getInputs().add(new MutableObject<>(currentTopOp));
context.computeAndSetTypeEnvironmentForOperator(newOp);
@@ -681,34 +900,34 @@
}
}
- private static AbstractFunctionCallExpression getClosedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
- int position, SourceLocation sourceLoc) {
- Mutable<ILogicalExpression> indexRef =
- new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(position))));
- ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
- FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
- fnExpr.setSourceLocation(sourceLoc);
- return fnExpr;
- }
+ private AbstractFunctionCallExpression getFieldAccessFunction(Mutable<ILogicalExpression> varRef, int fieldPos,
+ List<String> fieldName) {
+ if (fieldName.size() == 1 && fieldPos != -1) {
+ Mutable<ILogicalExpression> indexRef =
+ new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt32(fieldPos))));
+ ScalarFunctionCallExpression fnExpr = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
+ fnExpr.setSourceLocation(sourceLoc);
+ return fnExpr;
- private static AbstractFunctionCallExpression getOpenOrNestedFieldAccessFunction(Mutable<ILogicalExpression> varRef,
- List<String> fields, SourceLocation sourceLoc) {
- ScalarFunctionCallExpression func;
- if (fields.size() > 1) {
- IAObject fieldList = stringListToAOrderedList(fields);
- Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
- // Create an expression for the nested case
- func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED),
- varRef, fieldRef);
} else {
- IAObject fieldList = new AString(fields.get(0));
- Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
- // Create an expression for the open field case (By name)
- func = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME),
- varRef, fieldRef);
+ ScalarFunctionCallExpression func;
+ if (fieldName.size() > 1) {
+ IAObject fieldList = stringListToAOrderedList(fieldName);
+ Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
+ // Create an expression for the nested case
+ func = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_NESTED), varRef, fieldRef);
+ } else {
+ IAObject fieldList = new AString(fieldName.get(0));
+ Mutable<ILogicalExpression> fieldRef = constantToMutableLogicalExpression(fieldList);
+ // Create an expression for the open field case (By name)
+ func = new ScalarFunctionCallExpression(
+ FunctionUtil.getFunctionInfo(BuiltinFunctions.FIELD_ACCESS_BY_NAME), varRef, fieldRef);
+ }
+ func.setSourceLocation(sourceLoc);
+ return func;
}
- func.setSourceLocation(sourceLoc);
- return func;
}
private static AOrderedList stringListToAOrderedList(List<String> fields) {
@@ -724,8 +943,7 @@
}
private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
- IVariableTypeEnvironment typeEnv, boolean forceFilter, SourceLocation sourceLoc)
- throws AlgebricksException {
+ IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
// Add 'is not null' to all nullable secondary index keys as a filtering
// condition.
@@ -763,6 +981,142 @@
return filterExpression;
}
+ /**
+ * Builds the nested plan required for array index maintenance.
+ */
+ private class UnnestBranchCreator implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+ private final List<LogicalVariable> lastFieldVars;
+ private LogicalVariable lastRecordVar;
+ private ILogicalOperator currentTop, currentBottom;
+ private boolean isFirstWalk = true;
+
+ public UnnestBranchCreator(LogicalVariable recordVar, ILogicalOperator sourceOperator) {
+ this.lastRecordVar = recordVar;
+ this.currentTop = sourceOperator;
+ this.lastFieldVars = new ArrayList<>();
+ }
+
+ public ILogicalPlan buildBranch() {
+ return new ALogicalPlanImpl(new MutableObject<>(currentTop));
+ }
+
+ public void lowerIsFirstWalkFlag() {
+ isFirstWalk = false;
+ }
+
+ public VariableReferenceExpression createLastRecordVarRef() {
+ VariableReferenceExpression varRef = new VariableReferenceExpression(lastRecordVar);
+ varRef.setSourceLocation(sourceLoc);
+ return varRef;
+ }
+
+ @SafeVarargs
+ public final void applyProjectDistinct(List<Mutable<ILogicalExpression>>... auxiliaryExpressions)
+ throws AlgebricksException {
+ // Apply the following: PROJECT(SK, AK) --> (ORDER (SK, AK)) implicitly --> DISTINCT (SK, AK) .
+ List<LogicalVariable> projectVars = new ArrayList<>(this.lastFieldVars);
+ List<Mutable<ILogicalExpression>> distinctVarRefs =
+ OperatorManipulationUtil.createVariableReferences(projectVars, sourceLoc);
+
+ // If we have any additional expressions to be added to our index, append them here.
+ if (auxiliaryExpressions.length > 0) {
+ for (List<Mutable<ILogicalExpression>> exprList : auxiliaryExpressions) {
+ if (exprList != null) {
+ // Sanity check: ensure that we only have variable references.
+ if (exprList.stream().anyMatch(
+ e -> !e.getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE))) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ "Given auxiliary expression list contains non-variable reference expressions. We"
+ + " cannot apply DISTINCT to this expression at this stage.");
+ }
+
+ distinctVarRefs.addAll(OperatorManipulationUtil.cloneExpressions(exprList));
+ for (Mutable<ILogicalExpression> e : OperatorManipulationUtil.cloneExpressions(exprList)) {
+ projectVars.add(((VariableReferenceExpression) e.getValue()).getVariableReference());
+ }
+ }
+ }
+ }
+
+ ProjectOperator projectOperator = new ProjectOperator(projectVars);
+ projectOperator.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, projectOperator, true);
+ DistinctOperator distinctOperator = new DistinctOperator(distinctVarRefs);
+ distinctOperator.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, distinctOperator, true);
+ }
+
+ public void applyFilteringExpression(Mutable<ILogicalExpression> filterExpression) throws AlgebricksException {
+ SelectOperator selectOperator = new SelectOperator(filterExpression, false, null);
+ selectOperator.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, selectOperator, true);
+ }
+
+ @Override
+ public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+ List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+ boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
+ if (!isFirstWalk) {
+ // We have already built the UNNEST path, do not build again.
+ return;
+ }
+
+ ILogicalExpression accessToUnnestVar;
+ if (isFirstUnnestInStep) {
+ // This is the first UNNEST step. Get the field we want to UNNEST from our record.
+ accessToUnnestVar = (startingStepRecordType != null)
+ ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
+ startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+ : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
+ } else {
+ // This is the second+ UNNEST step. Refer back to the previously unnested variable.
+ accessToUnnestVar = new VariableReferenceExpression(this.lastFieldVars.get(0));
+ this.lastFieldVars.clear();
+ }
+ UnnestingFunctionCallExpression scanCollection = new UnnestingFunctionCallExpression(
+ BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SCAN_COLLECTION),
+ Collections.singletonList(new MutableObject<>(accessToUnnestVar)));
+ scanCollection.setReturnsUniqueValues(false);
+ scanCollection.setSourceLocation(sourceLoc);
+ LogicalVariable unnestVar = context.newVar();
+ this.lastFieldVars.add(unnestVar);
+
+ UnnestOperator unnestOp = new UnnestOperator(unnestVar, new MutableObject<>(scanCollection));
+ unnestOp.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, unnestOp, true);
+ if (isFirstArrayStep) {
+ this.currentBottom = unnestOp;
+ }
+
+ if (isLastUnnestInIntermediateStep) {
+ // This is the last UNNEST before the next array step. Update our record variable.
+ this.lastRecordVar = unnestVar;
+ this.lastFieldVars.clear();
+ }
+ }
+
+ @Override
+ public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+ boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+ // If the final value is nested inside a record, add an additional ASSIGN.
+ if (!isNonArrayStep) {
+ return;
+ }
+
+ // Create the function to access our final field.
+ AbstractFunctionCallExpression accessToFinalVar = (startingStepRecordType != null)
+ ? getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()),
+ startingStepRecordType.getFieldIndex(fieldName.get(0)), fieldName)
+ : getFieldAccessFunction(new MutableObject<>(createLastRecordVarRef()), -1, fieldName);
+
+ LogicalVariable finalVar = context.newVar();
+ this.lastFieldVars.add(finalVar);
+ AssignOperator assignOperator = new AssignOperator(finalVar, new MutableObject<>(accessToFinalVar));
+ assignOperator.setSourceLocation(sourceLoc);
+ this.currentTop = introduceNewOp(currentTop, assignOperator, true);
+ }
+ }
+
private final class IndexFieldId {
private final int indicator;
private final List<String> fieldName;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index b2fdb74..76b57ef 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -19,9 +19,7 @@
package org.apache.asterix.optimizer.rules.am;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
-import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -38,26 +36,23 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.MetadataUtil;
-import org.apache.asterix.om.base.AOrderedList;
-import org.apache.asterix.om.base.AString;
-import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
-import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
import org.apache.asterix.optimizer.base.AnalysisUtil;
import org.apache.asterix.optimizer.rules.am.OptimizableOperatorSubTree.DataSourceType;
+import org.apache.asterix.optimizer.rules.util.FullTextUtil;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -65,8 +60,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
@@ -81,7 +74,7 @@
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-import com.google.common.collect.ImmutableSet;
+import com.google.common.base.Strings;
/**
* Class that embodies the commonalities between rewrite rules for access
@@ -91,15 +84,6 @@
protected MetadataProvider metadataProvider;
- // Function Identifier sets that retain the original field variable through each function's arguments
- private final ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
- ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
- BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
- BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
- BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
- BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
- BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
-
public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
protected static void registerAccessMethod(IAccessMethod accessMethod,
@@ -215,7 +199,7 @@
/**
* Choose all indexes that match the given access method. These indexes will be used as index-search
* to replace the given predicates in a SELECT operator. Also, if there are multiple same type of indexes
- * on the same field, only of them will be chosen. Allowed cases (AccessMethod, IndexType) are:
+ * on the same field, only one of them will be chosen. Allowed cases (AccessMethod, IndexType) are:
* [BTreeAccessMethod , IndexType.BTREE], [RTreeAccessMethod , IndexType.RTREE],
* [InvertedIndexAccessMethod, IndexType.SINGLE_PARTITION_WORD_INVIX || SINGLE_PARTITION_NGRAM_INVIX ||
* LENGTH_PARTITIONED_WORD_INVIX || LENGTH_PARTITIONED_NGRAM_INVIX]
@@ -236,13 +220,34 @@
IAccessMethod chosenAccessMethod = amEntry.getKey();
Index chosenIndex = indexEntry.getKey();
IndexType indexType = chosenIndex.getIndexType();
- boolean isKeywordOrNgramIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
- || indexType == IndexType.SINGLE_PARTITION_WORD_INVIX
+ boolean isKeywordIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.SINGLE_PARTITION_WORD_INVIX;
+ boolean isNgramIndexChosen = indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX
|| indexType == IndexType.SINGLE_PARTITION_NGRAM_INVIX;
if ((chosenAccessMethod == BTreeAccessMethod.INSTANCE && indexType == IndexType.BTREE)
+ || (chosenAccessMethod == ArrayBTreeAccessMethod.INSTANCE && indexType == IndexType.ARRAY)
|| (chosenAccessMethod == RTreeAccessMethod.INSTANCE && indexType == IndexType.RTREE)
- || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isKeywordOrNgramIndexChosen)) {
+ // the inverted index will be utilized
+ // For Ngram, the full-text config used in the index and in the query are always the default one,
+ // so we don't check if the full-text config in the index and query match
+ //
+ // Note that the ngram index can be used in both
+ // 1) full-text ftcontains() function
+ // 2) non-full-text, regular string contains() function
+ // 3) edit-distance functions that take keyword as an argument,
+ // e.g. edit_distance_check() when the threshold is larger than 1
+ || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isNgramIndexChosen)
+ // the inverted index will be utilized
+ // For keyword, different full-text configs may apply to different indexes on the same field,
+ // so we need to check if the config used in the index matches the config in the ftcontains() query
+ // If not, then we cannot use this index.
+ //
+ // Note that for now, the keyword/fulltext index can be utilized in
+ // 1) the full-text ftcontains() function
+ // 2) functions that take keyword as an argument, e.g. edit_distance_check() when the threshold is 1
+ || (chosenAccessMethod == InvertedIndexAccessMethod.INSTANCE && isKeywordIndexChosen
+ && isSameFullTextConfigInIndexAndQuery(analysisCtx, chosenIndex.getIndexDetails()))) {
+
if (resultVarsToIndexTypesMap.containsKey(indexEntry.getValue())) {
List<IndexType> appliedIndexTypes = resultVarsToIndexTypesMap.get(indexEntry.getValue());
if (!appliedIndexTypes.contains(indexType)) {
@@ -261,6 +266,32 @@
return result;
}
+ private boolean isSameFullTextConfigInIndexAndQuery(AccessMethodAnalysisContext analysisCtx,
+ Index.IIndexDetails indexDetails) {
+ String indexFullTextConfig = ((Index.TextIndexDetails) indexDetails).getFullTextConfigName();
+
+ IOptimizableFuncExpr expr = analysisCtx.getMatchedFuncExpr(0);
+ if (FullTextUtil.isFullTextContainsFunctionExpr(expr)) {
+ // ftcontains()
+ String expectedConfig = FullTextUtil.getFullTextConfigNameFromExpr(expr);
+ if (Strings.isNullOrEmpty(expectedConfig)) {
+ return Strings.isNullOrEmpty(indexFullTextConfig);
+ } else if (expectedConfig.equals(indexFullTextConfig)) {
+ return true;
+ }
+ } else {
+ // besides ftcontains(), there are other functions that utilize the full-text inverted-index,
+ // e.g. edit_distance_check(),
+ // for now, we don't accept users to specify the full-text config in those functions,
+ // that means, we assume the full-text config used in those function is always the default one with the name null,
+ // and if the index full-text config name is also null, the index can be utilized
+ if (Strings.isNullOrEmpty(indexFullTextConfig)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
/**
* Removes irrelevant access methods candidates, based on whether the
* expressions in the query match those in the index. For example, some
@@ -280,6 +311,39 @@
while (indexExprAndVarIt.hasNext()) {
Map.Entry<Index, List<Pair<Integer, Integer>>> indexExprAndVarEntry = indexExprAndVarIt.next();
Index index = indexExprAndVarEntry.getKey();
+ IndexType indexType = index.getIndexType();
+ if (!accessMethod.matchIndexType(indexType)) {
+ indexExprAndVarIt.remove();
+ continue;
+ }
+ List<List<String>> keyFieldNames;
+ List<IAType> keyFieldTypes;
+ switch (Index.IndexCategory.of(indexType)) {
+ case ARRAY:
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ keyFieldNames = new ArrayList<>();
+ keyFieldTypes = new ArrayList<>();
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ List<String> project = e.getProjectList().get(i);
+ keyFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+ keyFieldTypes.add(e.getTypeList().get(i));
+ }
+ }
+ break;
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ keyFieldNames = valueIndexDetails.getKeyFieldNames();
+ keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ keyFieldNames = textIndexDetails.getKeyFieldNames();
+ keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, String.valueOf(indexType));
+ }
boolean allUsed = true;
int lastFieldMatched = -1;
@@ -287,9 +351,9 @@
// Used to keep track of matched expressions (added for prefix search)
int numMatchedKeys = 0;
- for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
- List<String> keyField = index.getKeyFieldNames().get(i);
- final IAType keyType = index.getKeyFieldTypes().get(i);
+ for (int i = 0; i < keyFieldNames.size(); i++) {
+ List<String> keyField = keyFieldNames.get(i);
+ final IAType keyType = keyFieldTypes.get(i);
boolean foundKeyField = false;
Iterator<Pair<Integer, Integer>> exprsAndVarIter = indexExprAndVarEntry.getValue().iterator();
while (exprsAndVarIter.hasNext()) {
@@ -584,15 +648,52 @@
List<Index> indexCandidates = new ArrayList<>();
// Add an index to the candidates if one of the indexed fields is fieldName
for (Index index : datasetIndexes) {
+ List<List<String>> keyFieldNames;
+ List<IAType> keyFieldTypes;
+ List<Integer> keySources;
+ boolean isOverridingKeyFieldTypes;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case ARRAY:
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ keyFieldNames = new ArrayList<>();
+ keyFieldTypes = new ArrayList<>();
+ keySources = new ArrayList<>();
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ List<String> project = e.getProjectList().get(i);
+ keyFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+ keyFieldTypes.add(e.getTypeList().get(i).getType());
+ keySources.add(e.getSourceIndicator());
+ }
+ }
+ isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
+ break;
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ keyFieldNames = valueIndexDetails.getKeyFieldNames();
+ keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+ keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+ isOverridingKeyFieldTypes = valueIndexDetails.isOverridingKeyFieldTypes();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ keyFieldNames = textIndexDetails.getKeyFieldNames();
+ keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+ keySources = textIndexDetails.getKeyFieldSourceIndicators();
+ isOverridingKeyFieldTypes = textIndexDetails.isOverridingKeyFieldTypes();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ String.valueOf(index.getIndexType()));
+ }
// Need to also verify the index is pending no op
- int keyIdx = index.getKeyFieldNames().indexOf(fieldName);
- List<Integer> keySources = index.getKeyFieldSourceIndicators();
+ int keyIdx = keyFieldNames.indexOf(fieldName);
if (keyIdx >= 0 && keySourceMatches(keySources, keyIdx, fieldSource)
&& index.getPendingOp() == MetadataUtil.PENDING_NO_OP) {
indexCandidates.add(index);
boolean isFieldTypeUnknown = fieldType == BuiltinType.AMISSING || fieldType == BuiltinType.ANY;
- if (isFieldTypeUnknown && (!index.isOverridingKeyFieldTypes() || index.isEnforced())) {
- IAType indexedType = index.getKeyFieldTypes().get(keyIdx);
+ if (isFieldTypeUnknown && (!isOverridingKeyFieldTypes || index.isEnforced())) {
+ IAType indexedType = keyFieldTypes.get(keyIdx);
optFuncExpr.setFieldType(varIdx, indexedType);
}
analysisCtx.addIndexExpr(matchedSubTree.getDataset(), index, matchedFuncExprIndex, varIdx);
@@ -684,9 +785,13 @@
varRef.setSourceLocation(unnestOp.getSourceLocation());
optFuncExpr.setLogicalExpr(funcVarIndex, varRef);
} else {
- fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
- funcVarIndex, optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(),
- subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
+ if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
+ subTree.setLastMatchedDataSourceVars(0, funcVarIndex);
+ }
+ fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0,
+ subTree.getRecordType(), funcVarIndex,
+ optFuncExpr.getFuncExpr().getArguments().get(funcVarIndex).getValue(), subTree.getMetaRecordType(),
+ datasetMetaVar, fieldSource, false);
if (fieldName.isEmpty()) {
return;
}
@@ -709,25 +814,40 @@
OptimizableOperatorSubTree subTree, int assignOrUnnestIndex, LogicalVariable datasetMetaVar,
IOptimizationContext context, List<Index> datasetIndexes, int optFuncExprIndex,
AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+ boolean doesArrayIndexQualify = context.getPhysicalOptimizationConfig().isArrayIndexEnabled()
+ && datasetIndexes.stream().anyMatch(i -> i.getIndexType() == IndexType.ARRAY)
+ && assignOrUnnestIndex == subTree.getAssignsAndUnnests().size() - 1;
List<LogicalVariable> varList = assignOp.getVariables();
MutableInt fieldSource = new MutableInt(0);
for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
LogicalVariable var = varList.get(varIndex);
int optVarIndex = optFuncExpr.findLogicalVar(var);
- // No matching var in optFuncExpr.
if (optVarIndex == -1) {
+ if (doesArrayIndexQualify && subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
+ // We may be able to apply an array index to this variable.
+ Triple<Integer, List<String>, IAType> fieldTriplet =
+ AccessMethodUtils.analyzeVarForArrayIndexes(assignOp, optFuncExpr, subTree, datasetMetaVar,
+ context, datasetIndexes, analysisCtx.getMatchedFuncExprs(), varIndex);
+ if (fieldTriplet != null && subTree.hasDataSource()) {
+ fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
+ optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue());
+ }
+ }
continue;
}
// At this point we have matched the optimizable func
// expr at optFuncExprIndex to an assigned variable.
// Remember matching subtree.
optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
+ if (subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
+ subTree.setLastMatchedDataSourceVars(varIndex, optVarIndex);
+ }
fieldSource.setValue(0);
- List<String> fieldName = getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex,
- subTree.getRecordType(), optVarIndex,
+ List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(optFuncExpr, subTree,
+ assignOrUnnestIndex, varIndex, subTree.getRecordType(), optVarIndex,
optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue(), subTree.getMetaRecordType(),
- datasetMetaVar, fieldSource);
+ datasetMetaVar, fieldSource, false);
IAType fieldType = (IAType) context.getOutputTypeEnvironment(assignOp).getVarType(var);
// Set the fieldName in the corresponding matched
@@ -823,220 +943,6 @@
}
/**
- * Returns the field name corresponding to the assigned variable at
- * varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
- * access function after following a set of allowed functions.
- *
- * @throws AlgebricksException
- */
- protected List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
- int opIndex, int assignVarIndex, ARecordType recordType, int funcVarIndex,
- ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar, MutableInt fieldSource)
- throws AlgebricksException {
- // Get expression corresponding to opVar at varIndex.
- AbstractLogicalExpression expr = null;
- AbstractFunctionCallExpression childFuncExpr = null;
- AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(opIndex);
- if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
- AssignOperator assignOp = (AssignOperator) op;
- expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
- // Can't get a field name from a constant expression. So, return null.
- if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
- return Collections.emptyList();
- }
- childFuncExpr = (AbstractFunctionCallExpression) expr;
- } else {
- UnnestOperator unnestOp = (UnnestOperator) op;
- expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
- if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return Collections.emptyList();
- }
- childFuncExpr = (AbstractFunctionCallExpression) expr;
- if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
- return Collections.emptyList();
- }
- expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
- }
- if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
- return Collections.emptyList();
- }
- AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
- FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
-
- boolean isByName = false;
- boolean isFieldAccess = false;
- String fieldName = null;
- List<String> nestedAccessFieldName = null;
- int fieldIndex = -1;
- if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
- fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
- if (fieldName == null) {
- return Collections.emptyList();
- }
- isFieldAccess = true;
- isByName = true;
- } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
- Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
- if (idx == null) {
- return Collections.emptyList();
- }
- fieldIndex = idx;
- isFieldAccess = true;
- } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
- ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
- if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return Collections.emptyList();
- }
- ConstantExpression constExpr = (ConstantExpression) nameArg;
- AOrderedList orderedNestedFieldName =
- (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
- nestedAccessFieldName = new ArrayList<>();
- for (int i = 0; i < orderedNestedFieldName.size(); i++) {
- nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
- }
- isFieldAccess = true;
- isByName = true;
- }
- if (isFieldAccess) {
- LogicalVariable sourceVar =
- ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
- if (sourceVar.equals(metaVar)) {
- fieldSource.setValue(1);
- } else {
- fieldSource.setValue(0);
- }
- if (optFuncExpr != null) {
- optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
- }
- int[] assignAndExpressionIndexes = null;
-
- //go forward through nested assigns until you find the relevant one
- for (int i = opIndex + 1; i < subTree.getAssignsAndUnnests().size(); i++) {
- AbstractLogicalOperator subOp = subTree.getAssignsAndUnnests().get(i);
- List<LogicalVariable> varList;
-
- if (subOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
- //Nested was an assign
- varList = ((AssignOperator) subOp).getVariables();
- } else if (subOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
- //Nested is not an assign
- varList = ((UnnestOperator) subOp).getVariables();
- } else {
- break;
- }
-
- //Go through variables in assign to check for match
- for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
- LogicalVariable var = varList.get(varIndex);
- ArrayList<LogicalVariable> parentVars = new ArrayList<>();
- expr.getUsedVariables(parentVars);
-
- if (parentVars.contains(var)) {
- //Found the variable we are looking for.
- //return assign and index of expression
- int[] returnValues = { i, varIndex };
- assignAndExpressionIndexes = returnValues;
- }
- }
- }
- if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
- //We found the nested assign
-
- //Recursive call on nested assign
- List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
- assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
- parentFuncExpr, metaType, metaVar, fieldSource);
-
- if (parentFieldNames.isEmpty()) {
- //Nested assign was not a field access.
- //We will not use index
- return Collections.emptyList();
- }
-
- if (!isByName) {
- IAType subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
- : recordType.getSubFieldType(parentFieldNames);
- // Sub-field type can be AUnionType in case if it's optional. Thus, needs to get the actual type.
- subFieldType = TypeComputeUtils.getActualType(subFieldType);
- if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
- throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
- ARecordType.class.getName());
- }
- fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
- }
- if (optFuncExpr != null) {
- optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
- }
- //add fieldName to the nested fieldName, return
- if (nestedAccessFieldName != null) {
- for (int i = 0; i < nestedAccessFieldName.size(); i++) {
- parentFieldNames.add(nestedAccessFieldName.get(i));
- }
- } else {
- parentFieldNames.add(fieldName);
- }
- return (parentFieldNames);
- }
-
- if (optFuncExpr != null) {
- optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
- }
- //no nested assign, we are at the lowest level.
- if (isByName) {
- if (nestedAccessFieldName != null) {
- return nestedAccessFieldName;
- }
- return new ArrayList<>(Arrays.asList(fieldName));
- }
- return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
- : recordType.getFieldNames()[fieldIndex]));
-
- }
-
- if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
- return Collections.emptyList();
- }
- // We use a part of the field in edit distance computation
- if (optFuncExpr != null
- && optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
- optFuncExpr.setPartialField(true);
- }
- // We expect the function's argument to be a variable, otherwise we
- // cannot apply an index.
- ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
- if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
- return Collections.emptyList();
- }
- LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
- // We look for the assign or unnest operator that produces curVar below
- // the current operator
- for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
- .size(); assignOrUnnestIndex++) {
- AbstractLogicalOperator curOp = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
- if (curOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
- AssignOperator assignOp = (AssignOperator) curOp;
- List<LogicalVariable> varList = assignOp.getVariables();
- for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
- LogicalVariable var = varList.get(varIndex);
- if (var.equals(curVar) && optFuncExpr != null) {
- optFuncExpr.setSourceVar(funcVarIndex, var);
- return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
- funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource);
- }
- }
- } else {
- UnnestOperator unnestOp = (UnnestOperator) curOp;
- LogicalVariable var = unnestOp.getVariable();
- if (var.equals(curVar)) {
- getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
- childFuncExpr, metaType, metaVar, fieldSource);
- }
- }
- }
- return Collections.emptyList();
- }
-
- /**
* Finds the field name of each variable in the ASSIGN or UNNEST operators of the sub-tree.
*/
protected void fillFieldNamesInTheSubTree(OptimizableOperatorSubTree subTree) throws AlgebricksException {
@@ -1060,9 +966,9 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
- subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
- fieldSource);
+ List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+ assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
+ subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
@@ -1075,8 +981,9 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0, subTree.getRecordType(),
- -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource);
+ fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, 0,
+ subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource,
+ false);
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
@@ -1102,9 +1009,9 @@
// funcVarIndex is not required. Thus, we set it to -1.
// optFuncExpr and parentFuncExpr are not required, too. Thus, we set them to null.
fieldSource.setValue(0);
- List<String> fieldName = getFieldNameFromSubTree(null, subTree, assignOrUnnestIndex, varIndex,
- subTree.getRecordType(), -1, null, subTree.getMetaRecordType(), datasetMetaVar,
- fieldSource);
+ List<String> fieldName = AccessMethodUtils.getFieldNameFromSubTree(null, subTree,
+ assignOrUnnestIndex, varIndex, subTree.getRecordType(), -1, null,
+ subTree.getMetaRecordType(), datasetMetaVar, fieldSource, false);
if (fieldName != null && !fieldName.isEmpty()) {
subTree.getVarsToFieldNameMap().put(var, fieldName);
}
@@ -1120,10 +1027,10 @@
subTree.getPrimaryKeyVars(null, primaryKeyVarList);
Index primaryIndex = getPrimaryIndexFromDataSourceScanOp(subTree.getDataSourceRef().getValue());
-
+ List<List<String>> keyFieldNames =
+ ((Index.ValueIndexDetails) primaryIndex.getIndexDetails()).getKeyFieldNames();
for (int i = 0; i < primaryKeyVarList.size(); i++) {
- subTree.getVarsToFieldNameMap().put(primaryKeyVarList.get(i),
- primaryIndex.getKeyFieldNames().get(i));
+ subTree.getVarsToFieldNameMap().put(primaryKeyVarList.get(i), keyFieldNames.get(i));
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index e003ea5..c78e89f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -20,6 +20,7 @@
package org.apache.asterix.optimizer.rules.am;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
@@ -27,6 +28,7 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Set;
+import java.util.Stack;
import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -41,9 +43,11 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IACursor;
import org.apache.asterix.om.base.IAObject;
@@ -59,6 +63,7 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy.TypeCastingMathFunctionType;
import org.apache.asterix.om.utils.ConstantExpressionUtil;
import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -72,6 +77,7 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
@@ -88,6 +94,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
@@ -95,6 +102,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.SplitOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
@@ -103,6 +111,8 @@
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import com.google.common.collect.ImmutableSet;
+
/**
* Static helper functions for rewriting plans using indexes.
*/
@@ -115,6 +125,15 @@
CONDITIONAL_SPLIT_VAR
}
+ // Function Identifier sets that retain the original field variable through each function's arguments
+ private final static ImmutableSet<FunctionIdentifier> funcIDSetThatRetainFieldName =
+ ImmutableSet.of(BuiltinFunctions.WORD_TOKENS, BuiltinFunctions.GRAM_TOKENS, BuiltinFunctions.SUBSTRING,
+ BuiltinFunctions.SUBSTRING_BEFORE, BuiltinFunctions.SUBSTRING_AFTER,
+ BuiltinFunctions.CREATE_POLYGON, BuiltinFunctions.CREATE_MBR, BuiltinFunctions.CREATE_RECTANGLE,
+ BuiltinFunctions.CREATE_CIRCLE, BuiltinFunctions.CREATE_LINE, BuiltinFunctions.CREATE_POINT,
+ BuiltinFunctions.NUMERIC_ADD, BuiltinFunctions.NUMERIC_SUBTRACT, BuiltinFunctions.NUMERIC_MULTIPLY,
+ BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
+
public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
List<Object> target) throws AlgebricksException {
ARecordType recordType = (ARecordType) itemType;
@@ -324,6 +343,9 @@
boolean primaryKeysOnly = isInvertedIndex(index);
if (!primaryKeysOnly) {
switch (index.getIndexType()) {
+ case ARRAY:
+ dest.addAll(KeyFieldTypeUtil.getArrayBTreeIndexKeyTypes(index, recordType, metaRecordType));
+ break;
case BTREE:
dest.addAll(KeyFieldTypeUtil.getBTreeIndexKeyTypes(index, recordType, metaRecordType));
break;
@@ -952,10 +974,40 @@
private static AbstractUnnestMapOperator createFinalNonIndexOnlySearchPlan(Dataset dataset,
ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput,
- boolean retainMissing, boolean requiresBroadcast, List<LogicalVariable> primaryKeyVars,
- List<LogicalVariable> primaryIndexUnnestVars, List<Object> primaryIndexOutputTypes)
- throws AlgebricksException {
+ boolean retainMissing, boolean requiresBroadcast, boolean requiresDistinct,
+ List<LogicalVariable> primaryKeyVars, List<LogicalVariable> primaryIndexUnnestVars,
+ List<LogicalVariable> auxDistinctVars, List<Object> primaryIndexOutputTypes) throws AlgebricksException {
SourceLocation sourceLoc = inputOp.getSourceLocation();
+
+ // Sanity check: requiresDistinct and sortPrimaryKeys are mutually exclusive.
+ if (requiresDistinct && sortPrimaryKeys) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ "Non-index search plan " + "cannot include a DISTINCT and an ORDER.");
+ }
+
+ // If we have an array index, then we must only give unique keys to our primary-index scan.
+ DistinctOperator distinct = null;
+ if (requiresDistinct) {
+ List<Mutable<ILogicalExpression>> distinctExprs = new ArrayList<>();
+ for (LogicalVariable pkVar : primaryKeyVars) {
+ VariableReferenceExpression pkVarRef = new VariableReferenceExpression(pkVar);
+ pkVarRef.setSourceLocation(sourceLoc);
+ Mutable<ILogicalExpression> vRef = new MutableObject<>(pkVarRef);
+ distinctExprs.add(vRef);
+ }
+ for (LogicalVariable auxVar : auxDistinctVars) {
+ VariableReferenceExpression auxVarRef = new VariableReferenceExpression(auxVar);
+ auxVarRef.setSourceLocation(sourceLoc);
+ Mutable<ILogicalExpression> vRef = new MutableObject<>(auxVarRef);
+ distinctExprs.add(vRef);
+ }
+ distinct = new DistinctOperator(distinctExprs);
+ distinct.setSourceLocation(sourceLoc);
+ distinct.getInputs().add(new MutableObject<>(inputOp));
+ distinct.setExecutionMode(ExecutionMode.LOCAL);
+ context.computeAndSetTypeEnvironmentForOperator(distinct);
+ }
+
// Optionally add a sort on the primary-index keys before searching the primary index.
OrderOperator order = null;
if (sortPrimaryKeys) {
@@ -977,7 +1029,9 @@
AbstractUnnestMapOperator primaryIndexUnnestMapOp =
createPrimaryIndexUnnestMapOp(dataset, retainInput, retainMissing, requiresBroadcast, primaryKeyVars,
primaryIndexUnnestVars, primaryIndexOutputTypes, sourceLoc);
- if (sortPrimaryKeys) {
+ if (requiresDistinct) {
+ primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(distinct));
+ } else if (sortPrimaryKeys) {
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
} else {
primaryIndexUnnestMapOp.getInputs().add(new MutableObject<>(inputOp));
@@ -1020,7 +1074,8 @@
// key search (SK, PK) and those in the original plan (datasource scan).
LinkedHashMap<LogicalVariable, LogicalVariable> origVarToSIdxUnnestMapOpVarMap = new LinkedHashMap<>();
- List<List<String>> chosenIndexFieldNames = secondaryIndex.getKeyFieldNames();
+ Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+ List<List<String>> chosenIndexFieldNames = secondaryIndexDetails.getKeyFieldNames();
IndexType idxType = secondaryIndex.getIndexType();
// variables used in SELECT or JOIN operator
@@ -1549,7 +1604,7 @@
* (i.e., we can guarantee the correctness of the result.)
*
* Case A) non-index-only plan
- * sidx-search -> (optional) sort -> pdix-search
+ * sidx-search -> (optional) sort -> (optional) distinct -> pdix-search
*
* Case B) index-only plan
* left path (an instantTryLock() on the PK fail path):
@@ -1563,8 +1618,8 @@
Dataset dataset, ARecordType recordType, ARecordType metaRecordType, ILogicalOperator inputOp,
IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean retainMissing,
boolean requiresBroadcast, Index secondaryIndex, AccessMethodAnalysisContext analysisCtx,
- OptimizableOperatorSubTree subTree, LogicalVariable newMissingPlaceHolderForLOJ)
- throws AlgebricksException {
+ OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
+ LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
// Common part for the non-index-only plan and index-only plan
// Variables and types for the primary-index search.
List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<>();
@@ -1577,21 +1632,32 @@
List<LogicalVariable> pkVarsFromSIdxUnnestMapOp = AccessMethodUtils.getKeyVarsFromSecondaryUnnestMap(dataset,
recordType, metaRecordType, inputOp, secondaryIndex, SecondaryUnnestMapOutputVarType.PRIMARY_KEY);
- // Index-only plan or not?
+ // Index-only plan or not? Array-index involved or not?
boolean isIndexOnlyPlan = analysisCtx.getIndexOnlyPlanInfo().getFirst();
+ boolean isArrayIndex = secondaryIndex.getIndexType() == IndexType.ARRAY;
- // Non-index-only plan case: creates ORDER -> UNNEST-MAP(Primary-index search) and return that unnest-map op.
+ // Non-index-only plan case: creates (ORDER)? -> (DISTINCT)? -> UNNEST-MAP(PIDX) and return that unnest-map op.
if (!isIndexOnlyPlan) {
- return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, sortPrimaryKeys, retainInput,
- retainMissing, requiresBroadcast, pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars,
- primaryIndexOutputTypes);
- } else {
+ // If we have a join + an array index, we need add the join source PK to the DISTINCT + ORDER.
+ List<LogicalVariable> joinPKVars = Collections.emptyList();
+ if (isArrayIndex && probeSubTree != null) {
+ joinPKVars = probeSubTree.getDataSourceVariables().subList(0,
+ probeSubTree.getDataSourceVariables().size() - 1);
+ }
+
+ return createFinalNonIndexOnlySearchPlan(dataset, inputOp, context, !isArrayIndex && sortPrimaryKeys,
+ retainInput, retainMissing, requiresBroadcast, isArrayIndex, pkVarsFromSIdxUnnestMapOp,
+ primaryIndexUnnestVars, joinPKVars, primaryIndexOutputTypes);
+ } else if (!isArrayIndex) {
// Index-only plan case: creates a UNIONALL operator that has two paths after the secondary unnest-map op,
// and returns it.
return createFinalIndexOnlySearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignsBeforeTopOpRef,
dataset, recordType, metaRecordType, inputOp, context, retainInput, retainMissing,
- requiresBroadcast, secondaryIndex, analysisCtx, subTree, newMissingPlaceHolderForLOJ,
+ requiresBroadcast, secondaryIndex, analysisCtx, indexSubTree, newMissingPlaceHolderForLOJ,
pkVarsFromSIdxUnnestMapOp, primaryIndexUnnestVars, primaryIndexOutputTypes);
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, inputOp.getSourceLocation(),
+ "Cannot use index-only plan with array indexes.");
}
}
@@ -1890,19 +1956,19 @@
// Since index-only plan doesn't access the primary index, we can't get the actual value in this case.
// Also, if no-index-only option is given, we stop here to honor that request.
boolean noIndexOnlyPlanOption = !context.getPhysicalOptimizationConfig().isIndexOnly();
- // TODO: For the inverted index access-method cases only:
+ // TODO: For the inverted index / array index access-method cases only:
// Since an inverted index can contain multiple secondary key entries per one primary key,
// Index-only plan can't be applied. For example, suppose there are two entries (SK1, SK2) for one PK.
// Since we can't access <SK1, PK>, <SK2, PK> at the same time unless we use tryLock (we use instantTryLock),
// right now, we can't support an index-only plan on an inverted index.
// Once this issue is resolved, we can apply an index-only plan.
- // One additional condition:
+ // One additional condition for inverted indexes:
// Even if the above is resolved, if a secondary key field is used after
// SELECT or JOIN operator, this can't be qualified as an index-only plan since
// an inverted index contains a part of a field value, not all of it.
if (noIndexOnlyPlanOption || dataset.getDatasetType() == DatasetType.EXTERNAL || chosenIndex.isPrimaryIndex()
- || chosenIndex.isOverridingKeyFieldTypes() || chosenIndex.isEnforced()
- || isInvertedIndex(chosenIndex)) {
+ || chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() || chosenIndex.isEnforced()
+ || isInvertedIndex(chosenIndex) || chosenIndex.getIndexType() == IndexType.ARRAY) {
indexOnlyPlanInfo.setFirst(false);
return;
}
@@ -2002,7 +2068,8 @@
// assign or data-source-scan in the subtree and the field-name of those variables are only PK or SK.
// Needs to check whether variables from the given select (join) operator only contain SK and/or PK condition.
List<List<String>> pkFieldNames = dataset.getPrimaryKeys();
- List<List<String>> chosenIndexFieldNames = chosenIndex.getKeyFieldNames();
+ Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
+ List<List<String>> chosenIndexFieldNames = chosenIndexDetails.getKeyFieldNames();
List<LogicalVariable> chosenIndexVars = new ArrayList<>();
// Collects variables that contain a CONSTANT expression in ASSIGN operators in the subtree.
@@ -2026,7 +2093,7 @@
}
// For the composite index, a secondary-index search generates a superset of the results.
- if (chosenIndex.getKeyFieldNames().size() > 1 && indexApplicableVarFoundCount > 1) {
+ if (chosenIndexDetails.getKeyFieldNames().size() > 1 && indexApplicableVarFoundCount > 1) {
requireVerificationAfterSIdxSearch = true;
}
@@ -2674,4 +2741,446 @@
AbstractExpressionAnnotationWithIndexNames ann = optFuncExpr.getFuncExpr().getAnnotation(annClass);
return ann == null ? null : ann.getIndexNames();
}
+
+ /**
+ * Returns the field name corresponding to the assigned variable at
+ * varIndex. Returns Collections.emptyList() if the expr at varIndex does not yield to a field
+ * access function after following a set of allowed functions.
+ *
+ * @throws AlgebricksException
+ */
+ public static List<String> getFieldNameFromSubTree(IOptimizableFuncExpr optFuncExpr,
+ OptimizableOperatorSubTree subTree, int opIndex, int assignVarIndex, ARecordType recordType,
+ int funcVarIndex, ILogicalExpression parentFuncExpr, ARecordType metaType, LogicalVariable metaVar,
+ MutableInt fieldSource, boolean isUnnestOverVarAllowed) throws AlgebricksException {
+ // Get expression corresponding to opVar at varIndex.
+ AbstractLogicalExpression expr = null;
+ AbstractFunctionCallExpression childFuncExpr = null;
+ AbstractLogicalOperator op = subTree.getAssignsAndUnnests().get(opIndex);
+ if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assignOp = (AssignOperator) op;
+ expr = (AbstractLogicalExpression) assignOp.getExpressions().get(assignVarIndex).getValue();
+ // Can't get a field name from a constant expression. So, return null.
+ if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ return Collections.emptyList();
+ }
+ childFuncExpr = (AbstractFunctionCallExpression) expr;
+ } else {
+ UnnestOperator unnestOp = (UnnestOperator) op;
+ expr = (AbstractLogicalExpression) unnestOp.getExpressionRef().getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return Collections.emptyList();
+ }
+ childFuncExpr = (AbstractFunctionCallExpression) expr;
+ if (childFuncExpr.getFunctionIdentifier() != BuiltinFunctions.SCAN_COLLECTION) {
+ return Collections.emptyList();
+ }
+ expr = (AbstractLogicalExpression) childFuncExpr.getArguments().get(0).getValue();
+ }
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return Collections.emptyList();
+ }
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+
+ boolean isByName = false;
+ boolean isFieldAccess = false;
+ String fieldName = null;
+ List<String> nestedAccessFieldName = null;
+ int fieldIndex = -1;
+ if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_NAME) {
+ fieldName = ConstantExpressionUtil.getStringArgument(funcExpr, 1);
+ if (fieldName == null) {
+ return Collections.emptyList();
+ }
+ isFieldAccess = true;
+ isByName = true;
+ } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_BY_INDEX) {
+ Integer idx = ConstantExpressionUtil.getIntArgument(funcExpr, 1);
+ if (idx == null) {
+ return Collections.emptyList();
+ }
+ fieldIndex = idx;
+ isFieldAccess = true;
+ } else if (funcIdent == BuiltinFunctions.FIELD_ACCESS_NESTED) {
+ ILogicalExpression nameArg = funcExpr.getArguments().get(1).getValue();
+ if (nameArg.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return Collections.emptyList();
+ }
+ ConstantExpression constExpr = (ConstantExpression) nameArg;
+ AOrderedList orderedNestedFieldName =
+ (AOrderedList) ((AsterixConstantValue) constExpr.getValue()).getObject();
+ nestedAccessFieldName = new ArrayList<>();
+ for (int i = 0; i < orderedNestedFieldName.size(); i++) {
+ nestedAccessFieldName.add(((AString) orderedNestedFieldName.getItem(i)).getStringValue());
+ }
+ isFieldAccess = true;
+ isByName = true;
+ }
+ if (isFieldAccess) {
+ LogicalVariable sourceVar =
+ ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
+ if (sourceVar.equals(metaVar)) {
+ fieldSource.setValue(1);
+ } else {
+ fieldSource.setValue(0);
+ }
+ if (optFuncExpr != null) {
+ optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
+ }
+ int[] assignAndExpressionIndexes = null;
+
+ //go forward through nested assigns until you find the relevant one
+ for (int i = opIndex + 1; i < subTree.getAssignsAndUnnests().size(); i++) {
+ AbstractLogicalOperator subOp = subTree.getAssignsAndUnnests().get(i);
+ List<LogicalVariable> varList;
+
+ if (subOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ //Nested was an assign
+ varList = ((AssignOperator) subOp).getVariables();
+ } else if (subOp.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+ //Nested is not an assign
+ varList = ((UnnestOperator) subOp).getVariables();
+ } else {
+ break;
+ }
+
+ //Go through variables in assign to check for match
+ for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+ LogicalVariable var = varList.get(varIndex);
+ ArrayList<LogicalVariable> parentVars = new ArrayList<>();
+ expr.getUsedVariables(parentVars);
+
+ if (parentVars.contains(var)) {
+ //Found the variable we are looking for.
+ //return assign and index of expression
+ int[] returnValues = { i, varIndex };
+ assignAndExpressionIndexes = returnValues;
+ }
+ }
+ }
+ if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
+ //We found the nested assign
+
+ //Recursive call on nested assign
+ List<String> parentFieldNames = getFieldNameFromSubTree(optFuncExpr, subTree,
+ assignAndExpressionIndexes[0], assignAndExpressionIndexes[1], recordType, funcVarIndex,
+ parentFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+
+ boolean isPreviousOperatorLegalUnnest = isUnnestOverVarAllowed && subTree.getAssignsAndUnnests()
+ .get(assignAndExpressionIndexes[0]).getOperatorTag().equals(LogicalOperatorTag.UNNEST);
+ if (parentFieldNames.isEmpty() && !isPreviousOperatorLegalUnnest) {
+ //Nested assign was not a field access.
+ //We will not use index
+ return Collections.emptyList();
+ } else if (isPreviousOperatorLegalUnnest) {
+ parentFieldNames = new ArrayList<>();
+ }
+
+ if (!isByName) {
+ IAType subFieldType;
+ if (isUnnestOverVarAllowed && isPreviousOperatorLegalUnnest) {
+ // In the case of UNNESTing over a variable, we use the record type given by our caller instead.
+ subFieldType = sourceVar.equals(metaVar) ? metaType : recordType;
+ } else {
+ subFieldType = sourceVar.equals(metaVar) ? metaType.getSubFieldType(parentFieldNames)
+ : recordType.getSubFieldType(parentFieldNames);
+ // Sub-field type can be AUnionType in case if optional. Thus, needs to get the actual type.
+ subFieldType = TypeComputeUtils.getActualType(subFieldType);
+ if (subFieldType.getTypeTag() != ATypeTag.OBJECT) {
+ throw CompilationException.create(ErrorCode.TYPE_CONVERT, subFieldType,
+ ARecordType.class.getName());
+ }
+ }
+ fieldName = ((ARecordType) subFieldType).getFieldNames()[fieldIndex];
+
+ }
+ if (optFuncExpr != null) {
+ optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+ }
+ //add fieldName to the nested fieldName, return
+ if (nestedAccessFieldName != null) {
+ for (int i = 0; i < nestedAccessFieldName.size(); i++) {
+ parentFieldNames.add(nestedAccessFieldName.get(i));
+ }
+ } else {
+ parentFieldNames.add(fieldName);
+ }
+ return (parentFieldNames);
+ }
+
+ if (optFuncExpr != null) {
+ optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
+ }
+ //no nested assign, we are at the lowest level.
+ if (isByName) {
+ if (nestedAccessFieldName != null) {
+ return nestedAccessFieldName;
+ }
+ return new ArrayList<>(Arrays.asList(fieldName));
+ }
+ return new ArrayList<>(Arrays.asList(sourceVar.equals(metaVar) ? metaType.getFieldNames()[fieldIndex]
+ : recordType.getFieldNames()[fieldIndex]));
+
+ }
+
+ if (!funcIDSetThatRetainFieldName.contains(funcIdent)) {
+ return Collections.emptyList();
+ }
+ // We use a part of the field in edit distance computation
+ if (optFuncExpr != null
+ && optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.EDIT_DISTANCE_CHECK) {
+ optFuncExpr.setPartialField(true);
+ }
+ // We expect the function's argument to be a variable, otherwise we
+ // cannot apply an index.
+ ILogicalExpression argExpr = funcExpr.getArguments().get(0).getValue();
+ if (argExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return Collections.emptyList();
+ }
+ LogicalVariable curVar = ((VariableReferenceExpression) argExpr).getVariableReference();
+ // We look for the assign or unnest operator that produces curVar below
+ // the current operator
+ for (int assignOrUnnestIndex = opIndex + 1; assignOrUnnestIndex < subTree.getAssignsAndUnnests()
+ .size(); assignOrUnnestIndex++) {
+ AbstractLogicalOperator curOp = subTree.getAssignsAndUnnests().get(assignOrUnnestIndex);
+ if (curOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator assignOp = (AssignOperator) curOp;
+ List<LogicalVariable> varList = assignOp.getVariables();
+ for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+ LogicalVariable var = varList.get(varIndex);
+ if (var.equals(curVar) && optFuncExpr != null) {
+ optFuncExpr.setSourceVar(funcVarIndex, var);
+ return getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, varIndex, recordType,
+ funcVarIndex, childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ }
+ }
+ } else {
+ UnnestOperator unnestOp = (UnnestOperator) curOp;
+ LogicalVariable var = unnestOp.getVariable();
+ if (var.equals(curVar)) {
+ getFieldNameFromSubTree(optFuncExpr, subTree, assignOrUnnestIndex, 0, recordType, funcVarIndex,
+ childFuncExpr, metaType, metaVar, fieldSource, isUnnestOverVarAllowed);
+ }
+ }
+ }
+ return Collections.emptyList();
+ }
+
+ /**
+ * Determine whether an array index can be used for the given variable.
+ */
+ public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(AssignOperator assignOp,
+ IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree, LogicalVariable datasetMetaVar,
+ IOptimizationContext context, List<Index> datasetIndexes, List<IOptimizableFuncExpr> matchedFuncExprs,
+ int assignVarIndex) throws AlgebricksException {
+
+ for (Index index : datasetIndexes) {
+ if (index.getIndexType() != IndexType.ARRAY) {
+ continue;
+ }
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ if (e.getUnnestList().isEmpty()) {
+ // Ignore the atomic part of this index (these are handled by the caller).
+ continue;
+ }
+
+ // We have found the array field for an array index.
+ for (List<String> project : e.getProjectList()) {
+ List<String> flattenedFieldName =
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
+ List<Integer> arrayIndicator = ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project);
+
+ Triple<Integer, List<String>, IAType> fieldTriplet =
+ matchAssignFieldInUnnestAssignStack(assignOp.getVariables().get(assignVarIndex),
+ assignVarIndex, optFuncExpr, subTree, datasetMetaVar, context, arrayIndicator,
+ flattenedFieldName, arrayIndexDetails.isOverridingKeyFieldTypes());
+
+ // This specific field aligns with our array index.
+ if (fieldTriplet.first > -1) {
+ int optVarIndex = fieldTriplet.first;
+ List<String> fieldName = fieldTriplet.second;
+ IAType fieldType = fieldTriplet.third;
+
+ // Remember matching subtree.
+ optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
+ MutableInt fieldSource = new MutableInt(0);
+ optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+ optFuncExpr.setFieldType(optVarIndex, fieldType);
+ IAType type = (IAType) context.getOutputTypeEnvironment(subTree.getRoot())
+ .getVarType(optFuncExpr.getLogicalVar(optVarIndex));
+ optFuncExpr.setFieldType(optVarIndex, type);
+
+ return fieldTriplet;
+ }
+ }
+ }
+ }
+
+ return null;
+ }
+
+ /**
+ * @param assignVar Variable from lowest assign that we are trying to match (i.e. the first array step var).
+ * @param assignVarIndex Index of the variable from the lowest assign.
+ * @param optFuncExpr The function expression we are trying to optimize.
+ * @param subTree Subtree for the function expression {@code optFunExpr}.
+ * @param datasetMetaVar Meta-variable from our subtree, if any exist.
+ * @param context Context required to get the type of the found variable.
+ * @param indexArrayIndicators Depth indicators of index to match our unnest/assign stack to.
+ * @param indexFieldNames Field names of index to match our unnest/assign stack to.
+ * @param areFieldNamesInAssign True if we have an open index. False otherwise.
+ */
+ private static Triple<Integer, List<String>, IAType> matchAssignFieldInUnnestAssignStack(LogicalVariable assignVar,
+ int assignVarIndex, IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
+ LogicalVariable datasetMetaVar, IOptimizationContext context, List<Integer> indexArrayIndicators,
+ List<String> indexFieldNames, boolean areFieldNamesInAssign) throws AlgebricksException {
+ Triple<Integer, List<String>, IAType> resultantTriple = new Triple<>(-1, new ArrayList<>(), null);
+ final int optVarIndex = subTree.getLastMatchedDataSourceVars().second;
+ if (optVarIndex < 0) {
+ return resultantTriple;
+ }
+ final ILogicalExpression optVarExpr = optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue();
+ optFuncExpr.setLogicalExpr(optVarIndex, optVarExpr);
+
+ // Build our assign / unnest stack. Do not include the very last assign (this is handled in the parent).
+ int indexOfWorkingOp = subTree.getAssignsAndUnnests().size() - 1;
+ Stack<AbstractLogicalOperator> logicalOperatorStack = new Stack<>();
+ logicalOperatorStack.addAll(subTree.getAssignsAndUnnests().subList(0, indexOfWorkingOp));
+ if (logicalOperatorStack.empty()) {
+ return resultantTriple;
+ }
+
+ // Aggregate our record paths, and pair these with their respective array indexes.
+ Pair<List<List<String>>, List<Integer>> unnestPairs =
+ ArrayIndexUtil.unnestComplexRecordPath(indexFieldNames, indexArrayIndicators);
+ AbstractLogicalOperator workingOp = null;
+ List<String> fieldNameForWorkingUnnest;
+ MutableInt fieldSource = new MutableInt(0);
+ ARecordType workingRecordType = subTree.getRecordType();
+
+ // TODO: (GLENN) Refactor this to use ArrayIndexUtil.
+ // Iterate through our array index structure. We must match the depth and field names for the caller's variable
+ // to qualify for an array-index optimization.
+ LogicalVariable varFromParent = assignVar;
+ for (int pairsIndex = 0; pairsIndex < unnestPairs.first.size(); pairsIndex++) {
+ if (logicalOperatorStack.empty()) {
+ return resultantTriple;
+ }
+ workingOp = logicalOperatorStack.pop();
+
+ // Explore our UNNEST path.
+ if (unnestPairs.second.get(pairsIndex) > 0) {
+ for (int i = (pairsIndex == 0) ? 1 : 0; i < unnestPairs.first.get(pairsIndex).size(); i++) {
+ // Match our parent assign variable to a variable used in our working assign.
+ assignVarIndex = findAssignVarIndex(workingOp, varFromParent);
+ if (logicalOperatorStack.empty() || assignVarIndex == -1) {
+ return resultantTriple;
+ }
+ varFromParent = ((AssignOperator) workingOp).getVariables().get(assignVarIndex);
+ indexOfWorkingOp--;
+ workingOp = logicalOperatorStack.pop();
+ }
+
+ // Get the field name associated with the current UNNEST.
+ if (workingOp.getOperatorTag() != LogicalOperatorTag.UNNEST) {
+ return resultantTriple;
+ }
+ fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
+ workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
+
+ if (!fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
+ return resultantTriple;
+ }
+ resultantTriple.second.addAll(fieldNameForWorkingUnnest);
+
+ IAType typeIntermediate = workingRecordType.getSubFieldType(fieldNameForWorkingUnnest);
+ for (int i = 0; i < unnestPairs.second.get(pairsIndex); i++) {
+ // If we are working with a closed index, then update our record type. For open types, we do not
+ // need to do this as the field name is stored in the expression itself.
+ if (!areFieldNamesInAssign && pairsIndex != unnestPairs.first.size() - 1) {
+ typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
+ if (typeIntermediate == null) {
+ return resultantTriple;
+ }
+ }
+ boolean isIntermediateUnnestInPath = (i != unnestPairs.second.get(pairsIndex) - 1);
+ if (!areFieldNamesInAssign && !isIntermediateUnnestInPath) {
+ if (typeIntermediate.getTypeTag().equals(ATypeTag.OBJECT)) {
+ workingRecordType = (ARecordType) typeIntermediate;
+ } else if (!typeIntermediate.getTypeTag().isListType()) {
+ return resultantTriple;
+ }
+ }
+
+ // Update our parent variable. If we are in-between UNNESTs, we need to fetch the next UNNEST.
+ if (isIntermediateUnnestInPath) {
+ workingOp = logicalOperatorStack.pop();
+ indexOfWorkingOp--;
+ }
+ varFromParent = ((UnnestOperator) workingOp).getVariable();
+ }
+ } else if (pairsIndex != 0) {
+ // We have explored an UNNEST array-path previously, and must now match a field name.
+ AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
+ indexOfWorkingOp -= unnestPairs.first.get(pairsIndex).size();
+ for (assignVarIndex = 0; assignVarIndex < workingOpAsAssign.getVariables().size(); assignVarIndex++) {
+ // Iterate through each of our ASSIGN's field names, and try to match the index field names.
+ fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
+ workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
+
+ if (fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
+ resultantTriple.second.addAll(fieldNameForWorkingUnnest);
+ break;
+ }
+ }
+
+ // We have exhausted all of our ASSIGN variables, but have not matched the field name. Exit early.
+ if (assignVarIndex == workingOpAsAssign.getVariables().size()) {
+ return resultantTriple;
+ }
+ }
+
+ indexOfWorkingOp--;
+ }
+
+ // We have found an applicable array index. Determine our optFuncIndex and fieldType.
+ if (workingOp != null && workingOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
+ LogicalVariable matchedVar = workingOpAsAssign.getVariables().get(assignVarIndex);
+ if (optFuncExpr.findLogicalVar(matchedVar) > -1) {
+ resultantTriple.first = optFuncExpr.findLogicalVar(matchedVar);
+ resultantTriple.third = (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(matchedVar);
+ optFuncExpr.setSourceVar(resultantTriple.first, matchedVar);
+ }
+
+ } else if (workingOp != null) {
+ UnnestOperator workingOpAsUnnest = (UnnestOperator) workingOp;
+ resultantTriple.first = optFuncExpr.findLogicalVar(workingOpAsUnnest.getVariable());
+ resultantTriple.third =
+ (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(workingOpAsUnnest.getVariable());
+ optFuncExpr.setSourceVar(resultantTriple.first, workingOpAsUnnest.getVariable());
+ }
+
+ return resultantTriple;
+ }
+
+ private static int findAssignVarIndex(AbstractLogicalOperator workingOp, LogicalVariable varFromParentAssign) {
+ if (workingOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+ return -1;
+ }
+ AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
+
+ // Match our parent assign variable to a variable used in our working assign.
+ List<LogicalVariable> variablesUsedInWorkingAssign = new ArrayList<>();
+ for (Mutable<ILogicalExpression> assignExpr : workingOpAsAssign.getExpressions()) {
+ assignExpr.getValue().getUsedVariables(variablesUsedInWorkingAssign);
+ int pos = variablesUsedInWorkingAssign.indexOf(varFromParentAssign);
+ if (pos != -1) {
+ return pos;
+ }
+ }
+ return -1;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
new file mode 100644
index 0000000..44e4a18
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayBTreeAccessMethod.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+
+// TODO (GLENN): Refactor the BTreeAccessMethod class and this class to extend a new "AbstractBTreeAccessMethod" class.
+/**
+ * Class for helping rewrite rules to choose and apply array BTree indexes.
+ */
+public class ArrayBTreeAccessMethod extends BTreeAccessMethod {
+ public static final ArrayBTreeAccessMethod INSTANCE = new ArrayBTreeAccessMethod();
+
+ @Override
+ public boolean matchAllIndexExprs(Index index) {
+ // Similar to BTree "matchAllIndexExprs", we only require all expressions to be matched if this is a composite
+ // key index with an unknowable field.
+ return ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+ .map(e -> e.getProjectList().size()).reduce(0, Integer::sum) > 1 && hasUnknownableField(index);
+ }
+
+ @Override
+ public boolean matchPrefixIndexExprs(Index index) {
+ return !matchAllIndexExprs(index);
+ }
+
+ private boolean hasUnknownableField(Index index) {
+ if (index.isSecondaryIndex() && index.getIndexDetails().isOverridingKeyFieldTypes() && !index.isEnforced()) {
+ return true;
+ }
+ for (Index.ArrayIndexElement e : ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ if (NonTaggedFormatUtil.isOptional(e.getTypeList().get(i))) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public ILogicalOperator createIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+ Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+ List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+ OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
+ LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+
+ Index.ArrayIndexDetails chosenIndexDetails = (Index.ArrayIndexDetails) chosenIndex.getIndexDetails();
+ List<List<String>> chosenIndexKeyFieldNames = new ArrayList<>();
+ List<IAType> chosenIndexKeyFieldTypes = new ArrayList<>();
+ List<Integer> chosenIndexKeyFieldSourceIndicators = new ArrayList<>();
+ for (Index.ArrayIndexElement e : chosenIndexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ chosenIndexKeyFieldNames
+ .add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)));
+ chosenIndexKeyFieldTypes.add(e.getTypeList().get(i));
+ chosenIndexKeyFieldSourceIndicators.add(e.getSourceIndicator());
+ }
+ }
+
+ return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
+ probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
+ newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
+ chosenIndexKeyFieldSourceIndicators);
+ }
+
+ @Override
+ protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
+ // TODO (GLENN): This assumes a flattened key list. Refactor / clarify this when removing depth indicators.
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) chosenIndexDetails;
+ int elementPos = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ if (elementPos == keyPos) {
+ return e.getTypeList().get(i);
+ }
+ elementPos++;
+ }
+ }
+
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "No array index element found, but using " + "an array access method.");
+ }
+
+ @Override
+ public boolean matchIndexType(IndexType indexType) {
+ return indexType == IndexType.ARRAY;
+ }
+
+ @Override
+ public String getName() {
+ return "ARRAY_BTREE_ACCESS_METHOD";
+ }
+
+ @Override
+ public int compareTo(IAccessMethod o) {
+ return this.getName().compareTo(o.getName());
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 21892cc..cad5f12 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -96,7 +96,8 @@
// That is, this function can produce false positive results if it is set to true.
// In this case, an index-search alone cannot replace the given SELECT condition and
// that SELECT condition needs to be applied after the index-search to get the correct results.
- // For B+Tree indexes, there are no false positive results unless the given index is a composite index.
+ // For B+Tree indexes, there are no false positive results unless the given index is a composite index or an array
+ // index.
private static final List<Pair<FunctionIdentifier, Boolean>> FUNC_IDENTIFIERS = Collections
.unmodifiableList(Arrays.asList(new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.EQ, false),
new Pair<FunctionIdentifier, Boolean>(AlgebricksBuiltinFunctions.LE, false),
@@ -127,7 +128,8 @@
public boolean matchAllIndexExprs(Index index) {
// require all expressions to be matched if this is a composite key index which has an unknownable key field.
// because we only add a tuple to the index if all its key fields are not null/missing.
- return index.getKeyFieldTypes().size() > 1 && hasUnknownableField(index);
+ return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes().size() > 1
+ && hasUnknownableField(index);
}
@Override
@@ -136,10 +138,10 @@
}
private boolean hasUnknownableField(Index index) {
- if (index.isSecondaryIndex() && index.isOverridingKeyFieldTypes() && !index.isEnforced()) {
+ if (index.isSecondaryIndex() && index.getIndexDetails().isOverridingKeyFieldTypes() && !index.isEnforced()) {
return true;
}
- for (IAType fieldType : index.getKeyFieldTypes()) {
+ for (IAType fieldType : ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
if (NonTaggedFormatUtil.isOptional(fieldType)) {
return true;
}
@@ -320,6 +322,26 @@
OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
LogicalVariable newMissingPlaceHolderForLOJ) throws AlgebricksException {
+
+ Index.ValueIndexDetails chosenIndexDetails = (Index.ValueIndexDetails) chosenIndex.getIndexDetails();
+ List<List<String>> chosenIndexKeyFieldNames = chosenIndexDetails.getKeyFieldNames();
+ List<IAType> chosenIndexKeyFieldTypes = chosenIndexDetails.getKeyFieldTypes();
+ List<Integer> chosenIndexKeyFieldSourceIndicators = chosenIndexDetails.getKeyFieldSourceIndicators();
+
+ return createBTreeIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef, assignBeforeTheOpRefs, indexSubTree,
+ probeSubTree, chosenIndex, analysisCtx, retainInput, retainMissing, requiresBroadcast, context,
+ newMissingPlaceHolderForLOJ, chosenIndexKeyFieldNames, chosenIndexKeyFieldTypes,
+ chosenIndexKeyFieldSourceIndicators);
+ }
+
+ protected ILogicalOperator createBTreeIndexSearchPlan(List<Mutable<ILogicalOperator>> afterTopOpRefs,
+ Mutable<ILogicalOperator> topOpRef, Mutable<ILogicalExpression> conditionRef,
+ List<Mutable<ILogicalOperator>> assignBeforeTheOpRefs, OptimizableOperatorSubTree indexSubTree,
+ OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+ boolean retainInput, boolean retainMissing, boolean requiresBroadcast, IOptimizationContext context,
+ LogicalVariable newMissingPlaceHolderForLOJ, List<List<String>> chosenIndexKeyFieldNames,
+ List<IAType> chosenIndexKeyFieldTypes, List<Integer> chosenIndexKeyFieldSourceIndicators)
+ throws AlgebricksException {
Dataset dataset = indexSubTree.getDataset();
ARecordType recordType = indexSubTree.getRecordType();
ARecordType metaRecordType = indexSubTree.getMetaRecordType();
@@ -370,12 +392,12 @@
for (Pair<Integer, Integer> exprIndex : exprAndVarList) {
// Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(exprIndex.first);
- int keyPos = indexOf(optFuncExpr.getFieldName(0), optFuncExpr.getFieldSource(0),
- chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
+ int keyPos = indexOf(optFuncExpr.getFieldName(0), optFuncExpr.getFieldSource(0), chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldSourceIndicators);
if (keyPos < 0 && optFuncExpr.getNumLogicalVars() > 1) {
// If we are optimizing a join, the matching field may be the second field name.
- keyPos = indexOf(optFuncExpr.getFieldName(1), optFuncExpr.getFieldSource(1),
- chosenIndex.getKeyFieldNames(), chosenIndex.getKeyFieldSourceIndicators());
+ keyPos = indexOf(optFuncExpr.getFieldName(1), optFuncExpr.getFieldSource(1), chosenIndexKeyFieldNames,
+ chosenIndexKeyFieldSourceIndicators);
}
if (keyPos < 0) {
throw CompilationException.create(ErrorCode.NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR,
@@ -385,7 +407,7 @@
// The second expression will not be null only if we are creating an EQ search predicate
// with a FLOAT or a DOUBLE constant that will be fed into an INTEGER index.
// This is required because of type-casting. Refer to AccessMethodUtils.createSearchKeyExpr for details.
- IAType indexedFieldType = chosenIndex.getKeyFieldTypes().get(keyPos);
+ IAType indexedFieldType = chosenIndexKeyFieldTypes.get(keyPos);
Triple<ILogicalExpression, ILogicalExpression, Boolean> returnedSearchKeyExpr =
AccessMethodUtils.createSearchKeyExpr(chosenIndex, optFuncExpr, indexedFieldType, probeSubTree);
ILogicalExpression searchKeyExpr = returnedSearchKeyExpr.first;
@@ -669,7 +691,7 @@
indexSearchOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef, conditionRef,
assignBeforeTheOpRefs, dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp,
context, true, retainInput, retainMissing, false, chosenIndex, analysisCtx, indexSubTree,
- newMissingPlaceHolderForLOJ);
+ probeSubTree, newMissingPlaceHolderForLOJ);
// Replaces the datasource scan with the new plan rooted at
// Get dataSourceRef operator -
@@ -895,7 +917,8 @@
return limit;
}
- private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType) {
+ private boolean relaxLimitTypeToInclusive(Index chosenIndex, int keyPos, boolean realTypeConvertedToIntegerType)
+ throws CompilationException {
// For a non-enforced index or an enforced index that stores a casted value on the given index,
// we need to apply the following transformation.
// For an index on a closed field, this transformation is not necessary since the value between
@@ -922,8 +945,8 @@
return true;
}
- if (chosenIndex.isOverridingKeyFieldTypes() && !chosenIndex.isEnforced()) {
- IAType indexedKeyType = chosenIndex.getKeyFieldTypes().get(keyPos);
+ if (chosenIndex.getIndexDetails().isOverridingKeyFieldTypes() && !chosenIndex.isEnforced()) {
+ IAType indexedKeyType = getIndexedKeyType(chosenIndex.getIndexDetails(), keyPos);
if (NonTaggedFormatUtil.isOptional(indexedKeyType)) {
indexedKeyType = ((AUnionType) indexedKeyType).getActualType();
}
@@ -943,6 +966,10 @@
return false;
}
+ protected IAType getIndexedKeyType(Index.IIndexDetails chosenIndexDetails, int keyPos) throws CompilationException {
+ return ((Index.ValueIndexDetails) chosenIndexDetails).getKeyFieldTypes().get(keyPos);
+ }
+
private boolean probeIsOnLhs(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree probeSubTree) {
if (probeSubTree == null) {
if (optFuncExpr.getConstantExpressions().length == 0) {
@@ -1011,6 +1038,11 @@
}
@Override
+ public boolean matchIndexType(IndexType indexType) {
+ return indexType == IndexType.BTREE;
+ }
+
+ @Override
public String getName() {
return "BTREE_ACCESS_METHOD";
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
index 84ee41e..b21cf12 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IAccessMethod.java
@@ -21,6 +21,7 @@
import java.util.Collection;
import java.util.List;
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.metadata.entities.Index;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -66,6 +67,14 @@
IOptimizationContext context, IVariableTypeEnvironment typeEnvironment) throws AlgebricksException;
/**
+ * Indicates whether this access method is applicable for the given index type.
+ *
+ * @return boolean
+ * @param indexType
+ */
+ public boolean matchIndexType(IndexType indexType);
+
+ /**
* Indicates whether all index expressions must be matched in order for this
* index to be applicable.
*
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 199f878..6a5964d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -94,6 +94,7 @@
protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
static {
+ registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
index bd5d82a..ab0ae5f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceLSMComponentFilterRule.java
@@ -34,6 +34,7 @@
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -121,7 +122,7 @@
for (int i = 0; i < analysisCtx.getMatchedFuncExprs().size(); i++) {
IOptimizableFuncExpr optFuncExpr = analysisCtx.getMatchedFuncExpr(i);
- boolean found = findMacthedExprFieldName(optFuncExpr, op, dataset, itemType, datasetIndexes, context,
+ boolean found = findMatchedExprFieldName(optFuncExpr, op, dataset, itemType, datasetIndexes, context,
filterSourceIndicator);
// the field name source should be consistent with the filter source indicator
if (found && optFuncExpr.getFieldName(0).equals(filterFieldName)
@@ -212,7 +213,9 @@
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
FunctionIdentifier fid = f.getFunctionIdentifier();
if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
- throw new IllegalStateException();
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ unnestMapOp.getSourceLocation(),
+ "Illegal function found, expected an " + "index-search.");
}
AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
jobGenParams.readFromFuncArgs(f.getArguments());
@@ -308,14 +311,17 @@
}
break;
case ORDER:
+ case DISTINCT:
ILogicalOperator child = intersectOrSortOrSplit.getValue().getInputs().get(0).getValue();
if (child.getOperatorTag().equals(LogicalOperatorTag.UNNEST_MAP)) {
UnnestMapOperator secondaryMap = (UnnestMapOperator) child;
- propagateFilterInSecondaryUnnsetMap(secondaryMap, filterType, context);
-
- setPrimaryFilterVar(primaryOp, secondaryMap.getPropagateIndexMinFilterVar(),
- secondaryMap.getPropagateIndexMaxFilterVar(), context);
+ // If we are already propagating our index filter, do not repeat this action.
+ if (!secondaryMap.propagateIndexFilter()) {
+ propagateFilterInSecondaryUnnsetMap(secondaryMap, filterType, context);
+ setPrimaryFilterVar(primaryOp, secondaryMap.getPropagateIndexMinFilterVar(),
+ secondaryMap.getPropagateIndexMaxFilterVar(), context);
+ }
}
break;
@@ -495,7 +501,7 @@
}
}
- private boolean findMacthedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
+ private boolean findMatchedExprFieldName(IOptimizableFuncExpr optFuncExpr, AbstractLogicalOperator op,
Dataset dataset, ARecordType filterSourceType, List<Index> datasetIndexes, IOptimizationContext context,
Integer filterSourceIndicator) throws AlgebricksException {
AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
@@ -557,7 +563,9 @@
AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
FunctionIdentifier fid = f.getFunctionIdentifier();
if (!fid.equals(BuiltinFunctions.INDEX_SEARCH)) {
- throw new IllegalStateException();
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ unnestMapOp.getSourceLocation(),
+ "Illegal function found, expected an " + "index-search.");
}
AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
jobGenParams.readFromFuncArgs(f.getArguments());
@@ -569,6 +577,11 @@
}
}
}
+ if (index == null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ unnestMapOp.getSourceLocation(),
+ "Could not find the corresponding index for an" + " index search.");
+ }
IAType metaItemType = ((MetadataProvider) context.getMetadataProvider())
.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
@@ -584,8 +597,40 @@
fieldName = dataset.getPrimaryKeys().get(idx);
keySource = getKeySource(DatasetUtil.getKeySourceIndicators(dataset), idx);
} else {
- fieldName = index.getKeyFieldNames().get(varIndex);
- keySource = getKeySource(index.getKeyFieldSourceIndicators(), varIndex);
+ List<List<String>> keyFieldNames;
+ List<Integer> keySources;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case ARRAY:
+ Index.ArrayIndexDetails arrayIndexDetails =
+ (Index.ArrayIndexDetails) index.getIndexDetails();
+ keyFieldNames = new ArrayList<>();
+ keySources = new ArrayList<>();
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ for (List<String> project : e.getProjectList()) {
+ keyFieldNames.add(
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+ keySources.add(e.getSourceIndicator());
+ }
+ }
+ break;
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails =
+ (Index.ValueIndexDetails) index.getIndexDetails();
+ keyFieldNames = valueIndexDetails.getKeyFieldNames();
+ keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails =
+ (Index.TextIndexDetails) index.getIndexDetails();
+ keyFieldNames = textIndexDetails.getKeyFieldNames();
+ keySources = textIndexDetails.getKeyFieldSourceIndicators();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ String.valueOf(index.getIndexType()));
+ }
+ fieldName = keyFieldNames.get(varIndex);
+ keySource = getKeySource(keySources, varIndex);
}
if (fieldName == null) {
return false;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
index 64f9068..d221fd1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroducePrimaryIndexForAggregationRule.java
@@ -284,7 +284,7 @@
// #2. get all indexes and look for the primary one
List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
for (Index index : indexes) {
- if (index.getKeyFieldNames().isEmpty()) {
+ if (index.isPrimaryKeyIndex()) {
return Pair.of(dataset, index);
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 4199ece..c64e517 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -30,6 +30,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.optimizer.rules.util.SelectInSubplanBranchCreator;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -122,11 +123,13 @@
protected IVariableTypeEnvironment typeEnvironment = null;
protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
protected List<Mutable<ILogicalOperator>> afterSelectRefs = null;
+ private final SelectInSubplanBranchCreator selectInSubplanBranchCreator = new SelectInSubplanBranchCreator();
// Register access methods.
protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<>();
static {
+ registerAccessMethod(ArrayBTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
@@ -370,6 +373,25 @@
analyzedAMs = new TreeMap<>();
}
+ // If there exists a SUBPLAN in our plan, and we are conditioning on a variable,
+ // attempt to rewrite this subplan to allow an array-index AM to be introduced.
+ // This rewrite is to be used **solely** for the purpose of changing a DATA-SCAN into a
+ // non-index-only plan branch. No nodes from this rewrite will be used beyond this point.
+ // If successful, this will create a non-index only plan that replaces the subplan's
+ // DATA-SCAN with a PIDX SEARCH <- DISTINCT <- ORDER <- SIDX SEARCH.
+ if (continueCheck && context.getPhysicalOptimizationConfig().isArrayIndexEnabled()) {
+ SelectOperator selectRewrite = selectInSubplanBranchCreator.createSelect(selectOp, context);
+ if (selectRewrite != null
+ && checkAndApplyTheSelectTransformation(new MutableObject<>(selectRewrite), context)) {
+ return true;
+
+ } else {
+ // If this optimization or temp-branch creation was not successful, restore our state.
+ selectRef = selectRefFromThisOp;
+ selectOp = selectInSubplanBranchCreator.getOriginalSelect();
+ }
+ }
+
// Check the condition of SELECT operator is a function call since
// only function call can be transformed using available indexes.
// If so, initialize the subtree information that will be used later to decide whether
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index 10fdfb9..8530dee 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -51,8 +51,7 @@
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.om.utils.ConstantExpressionUtil;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
+import org.apache.asterix.optimizer.rules.util.FullTextUtil;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -456,7 +455,7 @@
ILogicalOperator primaryIndexUnnestOp = AccessMethodUtils.createRestOfIndexSearchPlan(afterTopOpRefs, topOpRef,
conditionRef, assignBeforeTopOpRefs, dataSourceScan, dataset, recordType, metaRecordType,
secondaryIndexUnnestOp, context, true, retainInput, retainNull, false, chosenIndex, analysisCtx,
- indexSubTree, newNullPlaceHolderForLOJ);
+ indexSubTree, null, newNullPlaceHolderForLOJ);
return primaryIndexUnnestOp;
}
@@ -802,8 +801,8 @@
isFilterableArgs.add(new MutableObject<ILogicalExpression>(inputSearchVarRef));
// Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
isFilterableArgs.add(new MutableObject<ILogicalExpression>(optFuncExpr.getConstantExpr(0)));
- isFilterableArgs.add(new MutableObject<ILogicalExpression>(
- AccessMethodUtils.createInt32Constant(chosenIndex.getGramLength())));
+ isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(
+ ((Index.TextIndexDetails) chosenIndex.getIndexDetails()).getGramLength())));
boolean usePrePost = optFuncExpr.containsPartialField() ? false : true;
isFilterableArgs.add(
new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(usePrePost)));
@@ -887,7 +886,8 @@
jobGenParams.setSearchKeyType(typeTag);
}
- private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
+ private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams)
+ throws CompilationException {
if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == BuiltinFunctions.STRING_CONTAINS) {
jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
jobGenParams.setSimilarityThreshold(new AsterixConstantValue(AMissing.MISSING));
@@ -922,32 +922,12 @@
// We check the last argument of the given full-text search to see whether conjunctive or disjunctive
// search parameter is given. This is the last argument of the function call expression.
AbstractFunctionCallExpression funcExpr = optFuncExpr.getFuncExpr();
- jobGenParams.setSearchModifierType(getFullTextOption(funcExpr));
+ jobGenParams.setSearchModifierType(FullTextUtil.getFullTextSearchModeFromExpr(funcExpr));
jobGenParams.setSimilarityThreshold(new AsterixConstantValue(ANull.NULL));
}
}
- private static SearchModifierType getFullTextOption(AbstractFunctionCallExpression funcExpr) {
- if (funcExpr.getArguments().size() < 3 || funcExpr.getArguments().size() % 2 != 0) {
- // If no parameters or incorrect number of parameters are given, the default search type is returned.
- return SearchModifierType.CONJUNCTIVE;
- }
- // From the third argument, it contains full-text search options.
- for (int i = 2; i < funcExpr.getArguments().size(); i = i + 2) {
- String optionName = ConstantExpressionUtil.getStringArgument(funcExpr, i);
- if (optionName.equals(FullTextContainsDescriptor.SEARCH_MODE_OPTION)) {
- String searchType = ConstantExpressionUtil.getStringArgument(funcExpr, i + 1);
- if (searchType.equals(FullTextContainsDescriptor.CONJUNCTIVE_SEARCH_MODE_OPTION)) {
- return SearchModifierType.CONJUNCTIVE;
- } else {
- return SearchModifierType.DISJUNCTIVE;
- }
- }
- }
- return null;
- }
-
private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList,
ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context)
throws AlgebricksException {
@@ -997,7 +977,9 @@
private boolean isEditDistanceFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
if (index.isEnforced()) {
- return isEditDistanceFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+ return isEditDistanceFuncCompatible(
+ ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+ index.getIndexType());
} else {
return isEditDistanceFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
}
@@ -1046,13 +1028,14 @@
if (typeTag == ATypeTag.STRING) {
AString astr = (AString) listOrStrObj;
+ int gramLength = ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength();
// Compute merge threshold depending on the query grams contain pre- and postfixing
if (optFuncExpr.containsPartialField()) {
- mergeThreshold = (astr.getStringValue().length() - index.getGramLength() + 1)
- - edThresh.getIntegerValue() * index.getGramLength();
+ mergeThreshold =
+ (astr.getStringValue().length() - gramLength + 1) - edThresh.getIntegerValue() * gramLength;
} else {
- mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
- - edThresh.getIntegerValue() * index.getGramLength();
+ mergeThreshold =
+ (astr.getStringValue().length() + gramLength - 1) - edThresh.getIntegerValue() * gramLength;
}
}
@@ -1140,7 +1123,9 @@
private boolean isFullTextContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
if (index.isEnforced()) {
- return isFullTextContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+ return isFullTextContainsFuncCompatible(
+ ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+ index.getIndexType());
} else {
return isFullTextContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
}
@@ -1232,7 +1217,7 @@
// Check that the constant search string has at least gramLength characters.
if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
AString astr = (AString) strObj;
- if (astr.getStringValue().length() >= index.getGramLength()) {
+ if (astr.getStringValue().length() >= ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength()) {
return true;
}
}
@@ -1241,7 +1226,9 @@
private boolean isContainsFuncJoinOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
if (index.isEnforced()) {
- return isContainsFuncCompatible(index.getKeyFieldTypes().get(0).getTypeTag(), index.getIndexType());
+ return isContainsFuncCompatible(
+ ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes().get(0).getTypeTag(),
+ index.getIndexType());
} else {
return isContainsFuncCompatible(optFuncExpr.getFieldType(0).getTypeTag(), index.getIndexType());
}
@@ -1269,7 +1256,7 @@
boolean prePost = (searchModifierType == SearchModifierType.CONJUNCTIVE
|| searchModifierType == SearchModifierType.CONJUNCTIVE_EDIT_DISTANCE) ? false : true;
return BinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(searchKeyType,
- index.getGramLength(), prePost, false);
+ ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength(), prePost, false);
}
default: {
throw new CompilationException(ErrorCode.NO_TOKENIZER_FOR_TYPE, index.getIndexType());
@@ -1300,11 +1287,12 @@
switch (index.getIndexType()) {
case SINGLE_PARTITION_NGRAM_INVIX:
case LENGTH_PARTITIONED_NGRAM_INVIX: {
+ int gramLength = ((Index.TextIndexDetails) index.getIndexDetails()).getGramLength();
// Edit distance on strings, filtered with overlapping grams.
if (searchModifierType == SearchModifierType.EDIT_DISTANCE) {
- return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
+ return new EditDistanceSearchModifierFactory(gramLength, edThresh);
} else {
- return new ConjunctiveEditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
+ return new ConjunctiveEditDistanceSearchModifierFactory(gramLength, edThresh);
}
}
case SINGLE_PARTITION_WORD_INVIX:
@@ -1340,6 +1328,19 @@
}
@Override
+ public boolean matchIndexType(IndexType indexType) {
+ switch (indexType) {
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ return true;
+ default:
+ return false;
+ }
+ }
+
+ @Override
public String getName() {
return "INVERTED_INDEX_ACCESS_METHOD";
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index eeace15..d45e7f2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -72,6 +72,7 @@
private Mutable<ILogicalOperator> rootRef = null;
private final List<Mutable<ILogicalOperator>> assignsAndUnnestsRefs = new ArrayList<>();
private final List<AbstractLogicalOperator> assignsAndUnnests = new ArrayList<>();
+ private final Pair<Integer, Integer> lastMatchedDataSourceVars = new Pair<>(-1, -1);
private Mutable<ILogicalOperator> dataSourceRef = null;
private DataSourceType dataSourceType = DataSourceType.NO_DATASOURCE;
@@ -389,6 +390,8 @@
setRecordType(null);
setMetaRecordType(null);
setIxJoinOuterAdditionalRecordTypes(null);
+ lastMatchedDataSourceVars.first = -1;
+ lastMatchedDataSourceVars.second = -1;
}
/**
@@ -590,4 +593,12 @@
return varsToFieldNameMap;
}
+ public Pair<Integer, Integer> getLastMatchedDataSourceVars() {
+ return lastMatchedDataSourceVars;
+ }
+
+ public void setLastMatchedDataSourceVars(int varIndex, int optVarIndex) {
+ this.lastMatchedDataSourceVars.first = varIndex;
+ this.lastMatchedDataSourceVars.second = optVarIndex;
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index 80bf943..09575e2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -308,7 +308,7 @@
secondaryIndexUnnestOp, context, chosenIndex, retainInput, retainNull)
: AccessMethodUtils.createRestOfIndexSearchPlan(afterTopRefs, topRef, conditionRef, assignBeforeTopRefs,
dataSourceOp, dataset, recordType, metaRecordType, secondaryIndexUnnestOp, context, true,
- retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree,
+ retainInput, retainNull, false, chosenIndex, analysisCtx, indexSubTree, null,
newNullPlaceHolderForLOJ);
}
@@ -385,6 +385,11 @@
}
@Override
+ public boolean matchIndexType(IndexType indexType) {
+ return indexType == IndexType.RTREE;
+ }
+
+ @Override
public String getName() {
return "RTREE_ACCESS_METHOD";
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java
new file mode 100644
index 0000000..ab62e85
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/FullTextUtil.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.asterix.optimizer.rules.am.IOptimizableFuncExpr;
+import org.apache.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class FullTextUtil {
+
+ public static boolean isFullTextContainsFunctionExpr(IOptimizableFuncExpr expr) {
+ return isFullTextContainsFunctionExpr(expr.getFuncExpr());
+ }
+
+ public static boolean isFullTextContainsFunctionExpr(AbstractFunctionCallExpression expr) {
+ FunctionIdentifier funcId = expr.getFunctionIdentifier();
+ if (funcId.equals(BuiltinFunctions.FULLTEXT_CONTAINS)
+ || funcId.equals(BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION)) {
+ return true;
+ }
+ return false;
+ }
+
+ // If not a full-text function expression, then return null
+ // Otherwise, return the full-text config if one exists in the expression, otherwise return the default config
+ public static String getFullTextConfigNameFromExpr(IOptimizableFuncExpr expr) {
+ return getFullTextConfigNameFromExpr(expr.getFuncExpr());
+ }
+
+ // ToDo: here we are parsing the expr manually, maybe we can find a better way to parse the arguments,
+ // e.g. convert the argument into an AdmObjectNode and then read from the object node
+ public static String getFullTextConfigNameFromExpr(AbstractFunctionCallExpression funcExpr) {
+ if (isFullTextContainsFunctionExpr(funcExpr) == false) {
+ return null;
+ }
+
+ String configName = null;
+ List<Mutable<ILogicalExpression>> arguments = funcExpr.getArguments();
+
+ // The first two arguments are
+ // 1) the full-text record field to be queried,
+ // 2) the query keyword array
+ // The next fields are the list of full-text search options,
+ // say, the next 4 fields can be "mode", "all", "config", "my_full_text_config"
+ // Originally, the full-text search option is an Asterix record such as
+ // {"mode": "all", "config": "my_full_text_config"}
+ for (int i = 2; i < arguments.size(); i += 2) {
+ // The the full-text search option arguments are already checked in FullTextContainsParameterCheckAndSetRule,
+ String optionName = ConstantExpressionUtil.getStringConstant(arguments.get(i).getValue());
+
+ if (optionName.equalsIgnoreCase(FullTextContainsFunctionDescriptor.FULLTEXT_CONFIG_OPTION)) {
+ configName = ConstantExpressionUtil.getStringConstant(arguments.get(i + 1).getValue());
+ break;
+ }
+ }
+
+ return configName;
+ }
+
+ public static InvertedIndexAccessMethod.SearchModifierType getFullTextSearchModeFromExpr(
+ AbstractFunctionCallExpression funcExpr) {
+
+ // After the third argument, the following arguments are full-text search options.
+ for (int i = 2; i < funcExpr.getArguments().size(); i = i + 2) {
+ String optionName = ConstantExpressionUtil.getStringArgument(funcExpr, i);
+
+ if (optionName.equals(FullTextContainsFunctionDescriptor.SEARCH_MODE_OPTION)) {
+ String searchType = ConstantExpressionUtil.getStringArgument(funcExpr, i + 1);
+
+ if (searchType.equals(FullTextContainsFunctionDescriptor.SearchMode.ALL.getValue())) {
+ return InvertedIndexAccessMethod.SearchModifierType.CONJUNCTIVE;
+ } else {
+ return InvertedIndexAccessMethod.SearchModifierType.DISJUNCTIVE;
+ }
+ }
+ }
+
+ // Use CONJUNCTIVE by default
+ return InvertedIndexAccessMethod.SearchModifierType.CONJUNCTIVE;
+ }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
new file mode 100644
index 0000000..6efef16
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
@@ -0,0 +1,424 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.base.AInt16;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+/**
+ * For use in writing a "throwaway" branch which removes NTS and subplan operators. The result of this invocation is to
+ * be given to the {@code IntroduceSelectAccessMethodRule} to check if an array index can be used.
+ * <br>
+ * If we are given the pattern (an existential query):
+ * <pre>
+ * SELECT_1(some variable)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input) AGGREGATE(NON-EMPTY-STREAM)
+ * SELECT_2(some predicate)
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some predicate)
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * If we are given the pattern (a universal query):
+ * <pre>
+ * SELECT_1(some variable AND array is not empty)
+ * SUBPLAN_1 -------------------------------|
+ * (parent branch input) AGGREGATE(EMPTY-STREAM)
+ * SELECT_2(NOT(IF-MISSING-OR-NULL(some predicate)))
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * NESTED-TUPLE-SOURCE
+ * </pre>
+ * We return the following branch:
+ * <pre>
+ * SELECT_2(some predicate) <--- removed the NOT(IF-MISSING-OR-NULL(...))!
+ * (UNNEST/ASSIGN)*
+ * UNNEST(on variable)
+ * (parent branch input)
+ * </pre>
+ *
+ * In the case of nested-subplans, we return a copy of the innermost SELECT followed by all relevant UNNEST/ASSIGNs.
+ */
+public class SelectInSubplanBranchCreator {
+ private final static List<IAlgebricksConstantValue> zerosAsAsterixConstants =
+ Arrays.asList(new IAlgebricksConstantValue[] { new AsterixConstantValue(new AInt64(0)),
+ new AsterixConstantValue(new AInt32(0)), new AsterixConstantValue(new AInt16((short) 0)),
+ new AsterixConstantValue(new AInt8((byte) 0)) });
+
+ private IOptimizationContext context;
+ private SourceLocation sourceLocation;
+ private SelectOperator originalSelectRoot;
+
+ /**
+ * Create a new branch to match that of the form:
+ *
+ * <pre>
+ * SELECT (...)
+ * (UNNEST/ASSIGN)*
+ * UNNEST
+ * ...
+ * </pre>
+ *
+ * Operators are *created* here, rather than just reconnected from the original branch.
+ */
+ public SelectOperator createSelect(SelectOperator originalSelect, IOptimizationContext context)
+ throws AlgebricksException {
+ // Reset our context.
+ this.sourceLocation = originalSelect.getSourceLocation();
+ this.originalSelectRoot = originalSelect;
+ this.context = context;
+
+ // We expect a) a SUBPLAN as input to this SELECT, and b) our SELECT to be conditioning on a variable.
+ if (!originalSelect.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)
+ || !originalSelect.getCondition().getValue().getExpressionTag().equals(LogicalExpressionTag.VARIABLE)) {
+ return null;
+ }
+ LogicalVariable originalSelectVar =
+ ((VariableReferenceExpression) originalSelect.getCondition().getValue()).getVariableReference();
+
+ // Additionally, verify that the subplan does not produce any other variable other than the SELECT var above.
+ SubplanOperator subplanOperator = (SubplanOperator) originalSelect.getInputs().get(0).getValue();
+ List<LogicalVariable> subplanProducedVars = new ArrayList<>();
+ VariableUtilities.getProducedVariables(subplanOperator, subplanProducedVars);
+ if (subplanProducedVars.size() != 1 || !subplanProducedVars.get(0).equals(originalSelectVar)) {
+ return null;
+ }
+
+ return traverseSubplanBranch(subplanOperator);
+ }
+
+ /**
+ * To undo this process is to return what was passed to us at {@code createSelect} time.
+ */
+ public SelectOperator getOriginalSelect() {
+ return originalSelectRoot;
+ }
+
+ private SelectOperator traverseSubplanBranch(SubplanOperator subplanOperator) throws AlgebricksException {
+ // We only expect one plan, and one root.
+ if (subplanOperator.getNestedPlans().size() > 1
+ || subplanOperator.getNestedPlans().get(0).getRoots().size() > 1) {
+ return null;
+ }
+
+ // This root of our "subplan" should always be an aggregate.
+ ILogicalOperator workingSubplanRoot = subplanOperator.getNestedPlans().get(0).getRoots().get(0).getValue();
+ AggregateOperator workingSubplanRootAsAggregate;
+ if (!workingSubplanRoot.getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
+ return null;
+ }
+ workingSubplanRootAsAggregate = (AggregateOperator) workingSubplanRoot;
+
+ // Try to find a SELECT that we can optimize (i.e. has a function call).
+ SelectOperator optimizableSelect = null;
+ for (Mutable<ILogicalOperator> opInput : workingSubplanRoot.getInputs()) {
+ ILogicalOperator subplanOrSelect = findSubplanOrSelect(opInput.getValue());
+ if (subplanOrSelect == null) {
+ return null;
+
+ } else if (subplanOrSelect.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
+ optimizableSelect = traverseSubplanBranch((SubplanOperator) subplanOrSelect);
+
+ } else {
+ optimizableSelect = (SelectOperator) subplanOrSelect;
+ break;
+ }
+ }
+ if (optimizableSelect == null) {
+ return null;
+ }
+
+ // We have found a SELECT with a variable. Create a copy, and set this to our rewrite root.
+ SelectOperator newSelectOperator = new SelectOperator(optimizableSelect.getCondition(),
+ optimizableSelect.getRetainMissing(), optimizableSelect.getMissingPlaceholderVariable());
+
+ // Ensure that this SELECT represents a predicate for an existential query, and is a query we can optimize.
+ newSelectOperator = normalizeSelectCondition(workingSubplanRootAsAggregate, newSelectOperator,
+ subplanOperator.getInputs().get(0).getValue());
+ if (newSelectOperator == null) {
+ return null;
+ }
+ newSelectOperator.setSourceLocation(sourceLocation);
+ newSelectOperator.setExecutionMode(optimizableSelect.getExecutionMode());
+
+ // Follow this SELECT to the root of our nested-plan branch (i.e. the NESTED-TUPLE-SOURCE).
+ ILogicalOperator workingOriginalOperator = optimizableSelect, workingNewOperator = newSelectOperator;
+ UnnestOperator bottommostNewUnnest = null;
+ while (!workingOriginalOperator.getOperatorTag().equals(LogicalOperatorTag.NESTEDTUPLESOURCE)) {
+ if (workingOriginalOperator.getInputs().isEmpty()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ workingSubplanRoot.getSourceLocation(),
+ "NESTED-TUPLE-SOURCE expected in nested plan branch," + " but not found.");
+ }
+
+ switch (workingOriginalOperator.getOperatorTag()) {
+ case UNNEST:
+ UnnestOperator originalUnnest = (UnnestOperator) workingOriginalOperator;
+ UnnestOperator newUnnest =
+ new UnnestOperator(originalUnnest.getVariable(), originalUnnest.getExpressionRef());
+ newUnnest.setSourceLocation(sourceLocation);
+ workingNewOperator.getInputs().add(new MutableObject<>(newUnnest));
+ workingNewOperator = newUnnest;
+ bottommostNewUnnest = (UnnestOperator) workingNewOperator;
+ break;
+
+ case ASSIGN:
+ AssignOperator originalAssign = (AssignOperator) workingOriginalOperator;
+ AssignOperator newAssign =
+ new AssignOperator(originalAssign.getVariables(), originalAssign.getExpressions());
+ newAssign.setSourceLocation(sourceLocation);
+ workingNewOperator.getInputs().add(new MutableObject<>(newAssign));
+ workingNewOperator = newAssign;
+ break;
+
+ case SUBPLAN:
+ // TODO (GLENN): Work on supporting nested universal quantification.
+ return null;
+
+ case AGGREGATE:
+ case SELECT:
+ break;
+
+ default:
+ return null;
+ }
+
+ workingOriginalOperator = workingOriginalOperator.getInputs().get(0).getValue();
+ }
+
+ // If we are working with universal quantification, then we must also check whether or not we have a conjunct
+ // that asserts that the array should also be non-empty.
+ if (isUniversalQuantification(workingSubplanRootAsAggregate)
+ && !isArrayNonEmptyConjunctIncluded(bottommostNewUnnest, subplanOperator)) {
+ return null;
+ }
+
+ // We have added everything we need in our nested-plan branch. Now, connect the input of our SUBPLAN to our
+ // current working branch.
+ bottommostNewUnnest.getInputs().addAll(subplanOperator.getInputs());
+ OperatorManipulationUtil.computeTypeEnvironmentBottomUp(newSelectOperator, context);
+
+ return newSelectOperator;
+ }
+
+ private boolean isUniversalQuantification(AggregateOperator workingSubplanRoot) throws CompilationException {
+ AggregateFunctionCallExpression aggregateFunctionCallExpression =
+ (AggregateFunctionCallExpression) workingSubplanRoot.getExpressions().get(0).getValue();
+ if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.EMPTY_STREAM)) {
+ return true;
+ } else if (aggregateFunctionCallExpression.getFunctionIdentifier().equals(BuiltinFunctions.NON_EMPTY_STREAM)) {
+ return false;
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, workingSubplanRoot.getSourceLocation(),
+ "Unexpected aggregate function: " + aggregateFunctionCallExpression.getFunctionIdentifier());
+ }
+ }
+
+ private boolean isArrayNonEmptyConjunctIncluded(UnnestOperator firstUnnestInNTS, SubplanOperator subplanOperator) {
+ UnnestingFunctionCallExpression unnestFunction =
+ (UnnestingFunctionCallExpression) firstUnnestInNTS.getExpressionRef().getValue();
+ VariableReferenceExpression unnestVarExpr =
+ (VariableReferenceExpression) unnestFunction.getArguments().get(0).getValue();
+ LogicalVariable arrayVariable = unnestVarExpr.getVariableReference();
+
+ // TODO (GLENN): The SELECT directly below the SUBPLAN is the only operator we explore. This does not cover
+ // all predicates where the array may be non-empty (say, having an existential predicate located after this
+ // subplan).
+ if (!subplanOperator.getInputs().get(0).getValue().getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ return false;
+ }
+ SelectOperator subplanInputOperator = (SelectOperator) subplanOperator.getInputs().get(0).getValue();
+ ILogicalExpression selectCondExpr = subplanInputOperator.getCondition().getValue();
+ List<Mutable<ILogicalExpression>> conjunctsFromSelect = new ArrayList<>();
+ if (selectCondExpr.splitIntoConjuncts(conjunctsFromSelect)) {
+ // We have a collection of conjuncts. Analyze each conjunct w/ a function.
+ for (Mutable<ILogicalExpression> mutableConjuct : conjunctsFromSelect) {
+ ILogicalExpression workingConjunct = mutableConjuct.getValue();
+ if (workingConjunct.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && analyzeConjunctForArrayNonEmptiness(arrayVariable,
+ (ScalarFunctionCallExpression) workingConjunct)) {
+ return true;
+ }
+ }
+
+ // No such conjunct found.
+ return false;
+ }
+
+ if (!selectCondExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+ return false;
+ }
+ return analyzeConjunctForArrayNonEmptiness(arrayVariable, (ScalarFunctionCallExpression) selectCondExpr);
+ }
+
+ private boolean analyzeConjunctForArrayNonEmptiness(LogicalVariable arrayVariable,
+ ScalarFunctionCallExpression workingSelectCondExpr) {
+ // Handle the conjunct: LEN(arrayVar) > 0
+ if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.GT)) {
+ ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+ ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+ if (firstArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && ((ScalarFunctionCallExpression) firstArg).getFunctionIdentifier().equals(BuiltinFunctions.LEN)) {
+ ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) firstArg;
+ List<LogicalVariable> usedVariables = new ArrayList<>();
+ lenFunction.getUsedVariables(usedVariables);
+
+ return usedVariables.contains(arrayVariable)
+ && secondArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && zerosAsAsterixConstants.contains(((ConstantExpression) secondArg).getValue());
+ }
+ }
+
+ // Handle the conjunct: 0 < LEN(arrayVar)
+ else if (workingSelectCondExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)) {
+ ILogicalExpression firstArg = workingSelectCondExpr.getArguments().get(0).getValue();
+ ILogicalExpression secondArg = workingSelectCondExpr.getArguments().get(1).getValue();
+
+ if (secondArg.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)
+ && ((ScalarFunctionCallExpression) secondArg).getFunctionIdentifier()
+ .equals(BuiltinFunctions.LEN)) {
+ ScalarFunctionCallExpression lenFunction = (ScalarFunctionCallExpression) secondArg;
+ List<LogicalVariable> usedVariables = new ArrayList<>();
+ lenFunction.getUsedVariables(usedVariables);
+
+ return usedVariables.contains(arrayVariable)
+ && firstArg.getExpressionTag().equals(LogicalExpressionTag.CONSTANT)
+ && zerosAsAsterixConstants.contains(((ConstantExpression) firstArg).getValue());
+ }
+ }
+
+ // TODO (GLENN): Handle the cases 1) where the arrayVar is explicitly indexed, 2) the NOT function.
+ return false;
+ }
+
+ private SelectOperator normalizeSelectCondition(AggregateOperator aggregateOperator, SelectOperator selectOperator,
+ ILogicalOperator subplanInputOperator) throws AlgebricksException {
+ // The purpose of this function is to remove the NOT(IF-MISSING-OR-NULL(...)) functions for a universal
+ // quantification query. The {@code ArrayBTreeAccessMethod} does not recognize the former as optimizable
+ // functions, so we remove them here. This SELECT will never make it to the final query plan (after the
+ // {@code IntroduceSelectAccessMethodRule}), which allows us to get away with this logically incorrect branch.
+ if (!isUniversalQuantification(aggregateOperator)) {
+ // We are working with an existential quantification query. Do not modify the SELECT.
+ return selectOperator;
+
+ } else {
+ // We are working with a universal quantification query.
+ if (!subplanInputOperator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ return null;
+ }
+
+ ScalarFunctionCallExpression notFunction =
+ (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
+ if (!notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
+ return selectOperator;
+ }
+
+ ScalarFunctionCallExpression ifMissingOrNullFunction =
+ (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
+ if (!ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
+ return selectOperator;
+ }
+
+ Mutable<ILogicalExpression> newSelectCondition =
+ new MutableObject<>(ifMissingOrNullFunction.getArguments().get(0).getValue().cloneExpression());
+ return new SelectOperator(newSelectCondition, selectOperator.getRetainMissing(),
+ selectOperator.getMissingPlaceholderVariable());
+
+ }
+ }
+
+ private ILogicalOperator findSubplanOrSelect(ILogicalOperator operator) {
+ // We are trying to find a SELECT operator with a function call that is not "NOT(IF-MISSING-OR-NULL(...))".
+ if (operator.getOperatorTag().equals(LogicalOperatorTag.SELECT)) {
+ SelectOperator selectOperator = (SelectOperator) operator;
+ ILogicalExpression selectCondExpr = selectOperator.getCondition().getValue();
+ if (selectCondExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+
+ // Follow the chain of NOT(IF-MISSING-OR-NULL(...)) to see if we have a variable at the end.
+ ScalarFunctionCallExpression notFunction =
+ (ScalarFunctionCallExpression) selectOperator.getCondition().getValue();
+ if (notFunction.getFunctionIdentifier().equals(BuiltinFunctions.NOT)) {
+ ScalarFunctionCallExpression ifMissingOrNullFunction =
+ (ScalarFunctionCallExpression) notFunction.getArguments().get(0).getValue();
+ if (ifMissingOrNullFunction.getFunctionIdentifier().equals(BuiltinFunctions.IF_MISSING_OR_NULL)) {
+ ILogicalExpression finalExpr = ifMissingOrNullFunction.getArguments().get(0).getValue();
+ if (finalExpr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return selectOperator;
+ }
+ }
+
+ } else {
+ return selectOperator;
+ }
+ }
+ } else if (operator.getOperatorTag().equals(LogicalOperatorTag.SUBPLAN)) {
+ // We have found an additional SUBPLAN branch to explore. Recurse w/ caller function.
+ return operator;
+ }
+
+ // No matching operator found. Recurse on current operator input.
+ if (operator.getInputs().isEmpty()) {
+ return null;
+ } else {
+ return findSubplanOrSelect(operator.getInputs().get(0).getValue());
+ }
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
index b32b05e..13bcfb6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/ValidateUtil.java
@@ -27,7 +27,6 @@
import org.apache.asterix.metadata.utils.KeyFieldTypeUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -184,137 +183,111 @@
/**
* Validates the key fields that will be used as keys of an index.
*
- * @param recType
- * the record type
- * @param keyFieldNames
- * a map of key fields that will be validated
- * @param keyFieldTypes
- * a map of key types (if provided) that will be validated
* @param indexType
* the type of the index that its key fields is being validated
+ * @param fieldType
+ * a key field type
+ * @param displayFieldName
+ * a field name to use for error reporting
* @param sourceLoc
* the source location
* @throws AlgebricksException
*/
- public static void validateKeyFields(ARecordType recType, ARecordType metaRecType, List<List<String>> keyFieldNames,
- List<Integer> keySourceIndicators, List<IAType> keyFieldTypes, IndexType indexType,
+ public static void validateIndexFieldType(IndexType indexType, IAType fieldType, List<String> displayFieldName,
SourceLocation sourceLoc) throws AlgebricksException {
- List<IAType> fieldTypes =
- KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, keyFieldNames, keySourceIndicators);
- int pos = 0;
- boolean openFieldCompositeIdx = false;
- for (IAType fieldType : fieldTypes) {
- List<String> fieldName = keyFieldNames.get(pos);
- if (fieldType == null) {
- fieldType = keyFieldTypes.get(pos);
- if (keyFieldTypes.get(pos) == BuiltinType.AMISSING) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "A field with this name \"" + fieldName + "\" could not be found.");
+ switch (indexType) {
+ case ARRAY:
+ case BTREE:
+ switch (fieldType.getTypeTag()) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ case STRING:
+ case BINARY:
+ case DATE:
+ case TIME:
+ case DATETIME:
+ case UNION:
+ case UUID:
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the BTree index.");
}
- } else if (openFieldCompositeIdx) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "A closed field \"" + fieldName
- + "\" could be only in a prefix part of the composite index, containing opened field.");
- }
- if (keyFieldTypes.get(pos) != BuiltinType.AMISSING
- && fieldType.getTypeTag() != keyFieldTypes.get(pos).getTypeTag()) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "A field \"" + fieldName + "\" is already defined with the type \"" + fieldType + "\"");
- }
- switch (indexType) {
- case BTREE:
- switch (fieldType.getTypeTag()) {
- case TINYINT:
- case SMALLINT:
- case INTEGER:
- case BIGINT:
- case FLOAT:
- case DOUBLE:
- case STRING:
- case BINARY:
- case DATE:
- case TIME:
- case DATETIME:
- case UNION:
- case UUID:
- case YEARMONTHDURATION:
- case DAYTIMEDURATION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the BTree index.");
- }
- break;
- case RTREE:
- switch (fieldType.getTypeTag()) {
- case POINT:
- case LINE:
- case RECTANGLE:
- case CIRCLE:
- case POLYGON:
- case GEOMETRY:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the RTree index.");
- }
- break;
- case LENGTH_PARTITIONED_NGRAM_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the Length Partitioned N-Gram index.");
- }
- break;
- case LENGTH_PARTITIONED_WORD_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case MULTISET:
- case ARRAY:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the Length Partitioned Keyword index.");
- }
- break;
- case SINGLE_PARTITION_NGRAM_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the N-Gram index.");
- }
- break;
- case SINGLE_PARTITION_WORD_INVIX:
- switch (fieldType.getTypeTag()) {
- case STRING:
- case MULTISET:
- case ARRAY:
- case UNION:
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "The field \"" + fieldName + "\" which is of type " + fieldType.getTypeTag()
- + " cannot be indexed using the Keyword index.");
- }
- break;
- default:
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Invalid index type: " + indexType + ".");
- }
- pos++;
+ break;
+ case RTREE:
+ switch (fieldType.getTypeTag()) {
+ case POINT:
+ case LINE:
+ case RECTANGLE:
+ case CIRCLE:
+ case POLYGON:
+ case GEOMETRY:
+ case UNION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the RTree index.");
+ }
+ break;
+ case LENGTH_PARTITIONED_NGRAM_INVIX:
+ switch (fieldType.getTypeTag()) {
+ case STRING:
+ case UNION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the Length Partitioned N-Gram index.");
+ }
+ break;
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ switch (fieldType.getTypeTag()) {
+ case STRING:
+ case MULTISET:
+ case ARRAY:
+ case UNION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the Length Partitioned Keyword index.");
+ }
+ break;
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ switch (fieldType.getTypeTag()) {
+ case STRING:
+ case UNION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the N-Gram index.");
+ }
+ break;
+ case SINGLE_PARTITION_WORD_INVIX:
+ switch (fieldType.getTypeTag()) {
+ case STRING:
+ case MULTISET:
+ case ARRAY:
+ case UNION:
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "The field \"" + displayFieldName + "\" which is of type " + fieldType.getTypeTag()
+ + " cannot be indexed using the Keyword index.");
+ }
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, sourceLoc,
+ String.valueOf(indexType));
}
}
-
}
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json
new file mode 100644
index 0000000..4d66342
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-1.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "dates": [ "2016-04-26 19:49:16", "2016-08-30 18:36:57", "2016-10-15 02:45:18", "2016-11-18 01:54:50", "2017-04-20 18:39:06", "2017-05-03 17:58:02", "2019-03-19 22:04:48" ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "dates": [ "2018-05-25 19:52:07", "2018-09-18 16:09:44", "2019-10-18 21:29:09" ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "dates": [ "2019-06-07 17:54:58" ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "dates": [ "2011-05-03 20:54:05", "2011-08-23 20:49:45", "2014-12-04 06:13:01", "2016-11-16 19:25:55" ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "dates": [ "2016-06-18 21:35:45", "2016-10-15 18:17:51" ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "dates": [ "2015-04-02 21:45:17" ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "dates": [ "2014-08-27 17:49:18", "2015-12-19 21:30:31", "2018-11-27 15:53:50" ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "dates": [ "2014-01-28 20:56:04", "2014-11-16 16:11:58", "2015-11-15 19:21:53", "2015-11-15 19:33:39" ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "dates": [ "2019-04-11 18:30:12" ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "dates": [ "2015-06-06 20:01:06", "2019-03-14 22:01:52" ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "dates": [ "2018-09-29 18:55:17", "2018-10-20 16:48:05", "2018-10-20 22:20:24" ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "dates": [ "2011-04-23 21:11:22", "2014-05-04 19:42:48", "2014-05-11 19:16:08", "2014-06-04 19:14:18", "2015-12-05 19:22:42", "2017-05-15 23:19:00" ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "dates": [ "2012-07-13 21:43:57", "2016-12-24 02:27:31", "2017-08-31 00:35:26" ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "dates": [ "2013-04-13 12:35:33", "2013-08-19 23:35:49", "2013-10-04 19:14:56" ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "dates": [ "2016-09-10 19:26:19", "2018-09-08 14:15:37", "2019-09-13 22:47:25" ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "dates": [ "2017-05-19 14:30:16", "2017-05-19 14:30:25", "2017-08-28 15:49:37", "2017-09-20 20:19:51", "2017-10-01 16:31:05", "2017-10-01 16:56:27", "2017-12-27 23:33:20" ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "dates": [ "2019-06-05 18:22:49" ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "dates": [ "2011-09-24 21:37:32", "2014-03-10 20:20:07", "2015-05-27 00:40:24", "2015-08-29 17:58:15", "2018-03-16 15:03:26" ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "dates": [ "2015-03-16 23:51:16", "2015-12-21 04:48:01", "2016-10-28 20:22:42", "2016-10-28 20:23:00" ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "dates": [ "2013-10-22 16:49:21", "2014-11-21 17:39:24" ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "dates": [ "2014-08-07 18:30:48", "2014-09-16 20:41:45", "2014-10-12 23:22:27", "2015-07-21 20:43:56", "2015-07-21 20:45:07" ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "dates": [ "2015-05-02 19:49:05", "2015-05-06 03:52:18", "2015-09-26 01:13:19" ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "dates": [ "2015-04-11 13:14:14", "2015-11-21 16:05:56", "2016-05-06 14:10:04", "2017-08-09 15:15:10", "2017-10-21 15:12:56" ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "dates": [ "2015-12-03 18:44:00", "2016-03-17 18:19:21", "2016-11-02 15:58:38" ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "dates": [ "2019-04-04 22:02:37" ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "dates": [ "2019-02-27 14:03:08" ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "dates": [ "2011-12-21 19:02:51", "2012-04-15 04:21:39", "2012-04-15 14:23:56", "2013-06-30 22:39:51", "2013-10-04 20:34:13", "2014-07-16 02:28:40" ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "dates": [ "2018-06-13 20:16:07" ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "dates": [ "2015-05-29 16:46:17", "2015-06-01 15:03:53" ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "dates": [ "2011-10-08 12:02:23", "2014-08-18 02:11:11", "2016-01-07 05:27:51", "2016-10-21 20:15:55", "2016-12-01 03:57:10", "2016-12-29 01:54:42", "2018-07-22 19:55:31", "2018-09-07 01:42:54", "2019-03-08 03:41:06" ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json
new file mode 100644
index 0000000..2179005
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-2.json
@@ -0,0 +1,31 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": { "dates": [ "2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18", "2017-04-20", "2017-05-03", "2019-03-19" ], "times": [ "19:49:16", "18:36:57", "02:45:18", "01:54:50", "18:39:06", "17:58:02", "22:04:48" ] } }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": { "dates": [ "2018-05-25", "2018-09-18", "2019-10-18" ], "times": [ "19:52:07", "16:09:44", "21:29:09" ] } }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": { "dates": [ "2019-06-07" ], "times": [ "17:54:58" ] } }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": { "dates": [ "2011-05-03", "2011-08-23", "2014-12-04", "2016-11-16" ], "times": [ "20:54:05", "20:49:45", "06:13:01", "19:25:55" ] } }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": { "dates": [ "2016-06-18", "2016-10-15" ], "times": [ "21:35:45", "18:17:51" ] } }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": { "dates": [ "2015-04-02" ], "times": [ "21:45:17" ] } }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": { "dates": [ "2014-08-27", "2015-12-19", "2018-11-27" ], "times": [ "17:49:18", "21:30:31", "15:53:50" ] } }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": { "dates": [ "2014-01-28", "2014-11-16", "2015-11-15", "2015-11-15" ], "times": [ "20:56:04", "16:11:58", "19:21:53", "19:33:39" ] } }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": { "dates": [ "2019-04-11" ], "times": [ "18:30:12" ] } }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": { "dates": [ "2015-06-06", "2019-03-14" ], "times": [ "20:01:06", "22:01:52" ] } }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": { "dates": [ "2018-09-29", "2018-10-20", "2018-10-20" ], "times": [ "18:55:17", "16:48:05", "22:20:24" ] } }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": { "dates": [ "2011-04-23", "2014-05-04", "2014-05-11", "2014-06-04", "2015-12-05", "2017-05-15" ], "times": [ "21:11:22", "19:42:48", "19:16:08", "19:14:18", "19:22:42", "23:19:00" ] } }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": { "dates": [ "2012-07-13", "2016-12-24", "2017-08-31" ], "times": [ "21:43:57", "02:27:31", "00:35:26" ] } }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": { "dates": [ "2013-04-13", "2013-08-19", "2013-10-04" ], "times": [ "12:35:33", "23:35:49", "19:14:56" ] } }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": { "dates": [ "2016-09-10", "2018-09-08", "2019-09-13" ], "times": [ "19:26:19", "14:15:37", "22:47:25" ] } }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": { "dates": [ "2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27" ], "times": [ "14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20" ] } }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": { "dates": [ "2019-06-05" ], "times": [ "18:22:49" ] } }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": { "dates": [ "2011-09-24", "2014-03-10", "2015-05-27", "2015-08-29", "2018-03-16" ], "times": [ "21:37:32", "20:20:07", "00:40:24", "17:58:15", "15:03:26" ] } }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": { "dates": [ "2015-03-16", "2015-12-21", "2016-10-28", "2016-10-28" ], "times": [ "23:51:16", "04:48:01", "20:22:42", "20:23:00" ] } }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": { "dates": [ "2013-10-22", "2014-11-21" ], "times": [ "16:49:21", "17:39:24" ] } }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": { "dates": [ "2014-08-07", "2014-09-16", "2014-10-12", "2015-07-21", "2015-07-21" ], "times": [ "18:30:48", "20:41:45", "23:22:27", "20:43:56", "20:45:07" ] } }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": { "dates": [ "2015-05-02", "2015-05-06", "2015-09-26" ], "times": [ "19:49:05", "03:52:18", "01:13:19" ] } }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": { "dates": [ "2015-04-11", "2015-11-21", "2016-05-06", "2017-08-09", "2017-10-21" ], "times": [ "13:14:14", "16:05:56", "14:10:04", "15:15:10", "15:12:56" ] } }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": { "dates": [ "2015-12-03", "2016-03-17", "2016-11-02" ], "times": [ "18:44:00", "18:19:21", "15:58:38" ] } }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": { "dates": [ "2019-04-04" ], "times": [ "22:02:37" ] } }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": { "dates": [ "2019-02-27" ], "times": [ "14:03:08" ] } }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": { "dates": [ "2011-12-21", "2012-04-15", "2012-04-15", "2013-06-30", "2013-10-04", "2014-07-16" ], "times": [ "19:02:51", "04:21:39", "14:23:56", "22:39:51", "20:34:13", "02:28:40" ] } }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": { "dates": [ "2018-06-13" ], "times": [ "20:16:07" ] } }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": { "dates": [ "2015-05-29", "2015-06-01" ], "times": [ "16:46:17", "15:03:53" ] } }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": { "dates": [ "2011-10-08", "2014-08-18", "2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29", "2018-07-22", "2018-09-07", "2019-03-08" ], "times": [ "12:02:23", "02:11:11", "05:27:51", "20:15:55", "03:57:10", "01:54:42", "19:55:31", "01:42:54", "03:41:06" ] } }
+
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json
new file mode 100644
index 0000000..c5a711f
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-3.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": [ {"date": "2016-04-26", "time": "19:49:16"}, {"date": "2016-08-30", "time": "18:36:57"}, {"date": "2016-10-15", "time": "02:45:18"}, {"date": "2016-11-18", "time": "01:54:50"}, {"date": "2017-04-20", "time": "18:39:06"}, {"date": "2017-05-03", "time": "17:58:02"}, {"date": "2019-03-19", "time": "22:04:48"} ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": [ {"date": "2018-05-25", "time": "19:52:07"}, {"date": "2018-09-18", "time": "16:09:44"}, {"date": "2019-10-18", "time": "21:29:09"} ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": [ {"date": "2019-06-07", "time": "17:54:58"} ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": [ {"date": "2011-05-03", "time": "20:54:05"}, {"date": "2011-08-23", "time": "20:49:45"}, {"date": "2014-12-04", "time": "06:13:01"}, {"date": "2016-11-16", "time": "19:25:55"} ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": [ {"date": "2016-06-18", "time": "21:35:45"}, {"date": "2016-10-15", "time": "18:17:51"} ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": [ {"date": "2015-04-02", "time": "21:45:17"} ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": [ {"date": "2014-08-27", "time": "17:49:18"}, {"date": "2015-12-19", "time": "21:30:31"}, {"date": "2018-11-27", "time": "15:53:50"} ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": [ {"date": "2014-01-28", "time": "20:56:04"}, {"date": "2014-11-16", "time": "16:11:58"}, {"date": "2015-11-15", "time": "19:21:53"}, {"date": "2015-11-15", "time": "19:33:39"} ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": [ {"date": "2019-04-11", "time": "18:30:12"} ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": [ {"date": "2015-06-06", "time": "20:01:06"}, {"date": "2019-03-14", "time": "22:01:52"} ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": [ {"date": "2018-09-29", "time": "18:55:17"}, {"date": "2018-10-20", "time": "16:48:05"}, {"date": "2018-10-20", "time": "22:20:24"} ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": [ {"date": "2011-04-23", "time": "21:11:22"}, {"date": "2014-05-04", "time": "19:42:48"}, {"date": "2014-05-11", "time": "19:16:08"}, {"date": "2014-06-04", "time": "19:14:18"}, {"date": "2015-12-05", "time": "19:22:42"}, {"date": "2017-05-15", "time": "23:19:00"} ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": [ {"date": "2012-07-13", "time": "21:43:57"}, {"date": "2016-12-24", "time": "02:27:31"}, {"date": "2017-08-31", "time": "00:35:26"} ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": [ {"date": "2013-04-13", "time": "12:35:33"}, {"date": "2013-08-19", "time": "23:35:49"}, {"date": "2013-10-04", "time": "19:14:56"} ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": [ {"date": "2016-09-10", "time": "19:26:19"}, {"date": "2018-09-08", "time": "14:15:37"}, {"date": "2019-09-13", "time": "22:47:25"} ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": [ {"date": "2017-05-19", "time": "14:30:16"}, {"date": "2017-05-19", "time": "14:30:25"}, {"date": "2017-08-28", "time": "15:49:37"}, {"date": "2017-09-20", "time": "20:19:51"}, {"date": "2017-10-01", "time": "16:31:05"}, {"date": "2017-10-01", "time": "16:56:27"}, {"date": "2017-12-27", "time": "23:33:20"} ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": [ {"date": "2019-06-05", "time": "18:22:49"} ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": [ {"date": "2011-09-24", "time": "21:37:32"}, {"date": "2014-03-10", "time": "20:20:07"}, {"date": "2015-05-27", "time": "00:40:24"}, {"date": "2015-08-29", "time": "17:58:15"}, {"date": "2018-03-16", "time": "15:03:26"} ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": [ {"date": "2015-03-16", "time": "23:51:16"}, {"date": "2015-12-21", "time": "04:48:01"}, {"date": "2016-10-28", "time": "20:22:42"}, {"date": "2016-10-28", "time": "20:23:00"} ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": [ {"date": "2013-10-22", "time": "16:49:21"}, {"date": "2014-11-21", "time": "17:39:24"} ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": [ {"date": "2014-08-07", "time": "18:30:48"}, {"date": "2014-09-16", "time": "20:41:45"}, {"date": "2014-10-12", "time": "23:22:27"}, {"date": "2015-07-21", "time": "20:43:56"}, {"date": "2015-07-21", "time": "20:45:07"} ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": [ {"date": "2015-05-02", "time": "19:49:05"}, {"date": "2015-05-06", "time": "03:52:18"}, {"date": "2015-09-26", "time": "01:13:19"} ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": [ {"date": "2015-04-11", "time": "13:14:14"}, {"date": "2015-11-21", "time": "16:05:56"}, {"date": "2016-05-06", "time": "14:10:04"}, {"date": "2017-08-09", "time": "15:15:10"}, {"date": "2017-10-21", "time": "15:12:56"} ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": [ {"date": "2015-12-03", "time": "18:44:00"}, {"date": "2016-03-17", "time": "18:19:21"}, {"date": "2016-11-02", "time": "15:58:38"} ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": [ {"date": "2019-04-04", "time": "22:02:37"} ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": [ {"date": "2019-02-27", "time": "14:03:08"} ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": [ {"date": "2011-12-21", "time": "19:02:51"}, {"date": "2012-04-15", "time": "04:21:39"}, {"date": "2012-04-15", "time": "14:23:56"}, {"date": "2013-06-30", "time": "22:39:51"}, {"date": "2013-10-04", "time": "20:34:13"}, {"date": "2014-07-16", "time": "02:28:40"} ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": [ {"date": "2018-06-13", "time": "20:16:07"} ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": [ {"date": "2015-05-29", "time": "16:46:17"}, {"date": "2015-06-01", "time": "15:03:53"} ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": [ {"date": "2011-10-08", "time": "12:02:23"}, {"date": "2014-08-18", "time": "02:11:11"}, {"date": "2016-01-07", "time": "05:27:51"}, {"date": "2016-10-21", "time": "20:15:55"}, {"date": "2016-12-01", "time": "03:57:10"}, {"date": "2016-12-29", "time": "01:54:42"}, {"date": "2018-07-22", "time": "19:55:31"}, {"date": "2018-09-07", "time": "01:42:54"}, {"date": "2019-03-08", "time": "03:41:06"} ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json b/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json
new file mode 100644
index 0000000..ec90bb4
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/use-case-4.json
@@ -0,0 +1,30 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_times": [ { "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"], "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"] }, { "dates": ["2017-04-20", "2017-05-03"], "times": ["18:39:06", "17:58:02"] }, { "dates": ["2019-03-19"], "times": ["22:04:48"] } ] }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_times": [ { "dates": ["2018-05-25", "2018-09-18"], "times": ["19:52:07", "16:09:44"] }, { "dates": ["2019-10-18"], "times": ["21:29:09"] } ] }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_times": [ { "dates": ["2019-06-07"], "times": ["17:54:58"] } ] }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_times": [ { "dates": ["2011-05-03", "2011-08-23"], "times": ["20:54:05", "20:49:45"] }, { "dates": ["2014-12-04"], "times": ["06:13:01"] }, { "dates": ["2016-11-16"], "times": ["19:25:55"] } ] }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_times": [ { "dates": ["2016-06-18", "2016-10-15"], "times": ["21:35:45", "18:17:51"] } ] }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_times": [ { "dates": ["2015-04-02"], "times": ["21:45:17"] } ] }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_times": [ { "dates": ["2014-08-27"], "times": ["17:49:18"] }, { "dates": ["2015-12-19"], "times": ["21:30:31"] }, { "dates": ["2018-11-27"], "times": ["15:53:50"] } ] }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_times": [ { "dates": ["2014-01-28", "2014-11-16"], "times": ["20:56:04", "16:11:58"] }, { "dates": ["2015-11-15", "2015-11-15"], "times": ["19:21:53", "19:33:39"] } ] }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_times": [ { "dates": ["2019-04-11"], "times": ["18:30:12"] } ] }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_times": [ { "dates": ["2015-06-06"], "times": ["20:01:06"] }, { "dates": ["2019-03-14"], "times": ["22:01:52"] } ] }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_times": [ { "dates": ["2018-09-29", "2018-10-20", "2018-10-20"], "times": ["18:55:17", "16:48:05", "22:20:24"] } ] }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_times": [ { "dates": ["2011-04-23"], "times": ["21:11:22"] }, { "dates": ["2014-05-04", "2014-05-11", "2014-06-04"], "times": ["19:42:48", "19:16:08", "19:14:18"] }, { "dates": ["2015-12-05"], "times": ["19:22:42"] }, { "dates": ["2017-05-15"], "times": ["23:19:00"] } ] }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_times": [ { "dates": ["2012-07-13"], "times": ["21:43:57"] }, { "dates": ["2016-12-24"], "times": ["02:27:31"] }, { "dates": ["2017-08-31"], "times": ["00:35:26"] } ] }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_times": [ { "dates": ["2013-04-13", "2013-08-19", "2013-10-04"], "times": ["12:35:33", "23:35:49", "19:14:56"] } ] }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_times": [ { "dates": ["2016-09-10"], "times": ["19:26:19"] }, { "dates": ["2018-09-08"], "times": ["14:15:37"] }, { "dates": ["2019-09-13"], "times": ["22:47:25"] } ] }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_times": [ { "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"], "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"] } ] }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_times": [ { "dates": ["2019-06-05"], "times": ["18:22:49"] } ] }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_times": [ { "dates": ["2011-09-24"], "times": ["21:37:32"] }, { "dates": ["2014-03-10"], "times": ["20:20:07"] }, { "dates": ["2015-05-27", "2015-08-29"], "times": ["00:40:24", "17:58:15"] }, { "dates": ["2018-03-16"], "times": ["15:03:26"] } ] }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_times": [ { "dates": ["2015-03-16", "2015-12-21"], "times": ["23:51:16", "04:48:01"] }, { "dates": ["2016-10-28", "2016-10-28"], "times": ["20:22:42", "20:23:00"] } ] }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_times": [ { "dates": ["2013-10-22"], "times": ["16:49:21"] }, { "dates": ["2014-11-21"], "times": ["17:39:24"] } ] }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_times": [ { "dates": ["2014-08-07", "2014-09-16", "2014-10-12"], "times": ["18:30:48", "20:41:45", "23:22:27"] }, { "dates": ["2015-07-21", "2015-07-21"], "times": ["20:43:56", "20:45:07"] } ] }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_times": [ { "dates": ["2015-05-02", "2015-05-06", "2015-09-26"], "times": ["19:49:05", "03:52:18", "01:13:19"] } ] }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_times": [ { "dates": ["2015-04-11", "2015-11-21"], "times": ["13:14:14", "16:05:56"] }, { "dates": ["2016-05-06"], "times": ["14:10:04"] }, { "dates": ["2017-08-09", "2017-10-21"], "times": ["15:15:10", "15:12:56"] } ] }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_times": [ { "dates": ["2015-12-03"], "times": ["18:44:00"] }, { "dates": ["2016-03-17", "2016-11-02"], "times": ["18:19:21", "15:58:38"] } ] }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_times": [ { "dates": ["2019-04-04"], "times": ["22:02:37"] } ] }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_times": [ { "dates": ["2019-02-27"], "times": ["14:03:08"] } ] }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_times": [ { "dates": ["2011-12-21"], "times": ["19:02:51"] }, { "dates": ["2012-04-15", "2012-04-15"], "times": ["04:21:39", "14:23:56"] }, { "dates": ["2013-06-30", "2013-10-04"], "times": ["22:39:51", "20:34:13"] }, { "dates": ["2014-07-16"], "times": ["02:28:40"] } ] }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_times": [ { "dates": ["2018-06-13"], "times": ["20:16:07"] } ] }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_times": [ { "dates": ["2015-05-29", "2015-06-01"], "times": ["16:46:17", "15:03:53"] } ] }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_times": [ { "dates": ["2011-10-08"], "times": ["12:02:23"] }, { "dates": ["2014-08-18"], "times": ["02:11:11"] }, { "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"], "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"] }, { "dates": ["2018-07-22", "2018-09-07"], "times": ["19:55:31", "01:42:54"] }, { "dates": ["2019-03-08"], "times": ["03:41:06"] } ] }
diff --git a/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json b/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json
new file mode 100644
index 0000000..39505f0
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/with-3-level-record-path.json
@@ -0,0 +1,31 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18", "2017-04-20", "2017-05-03", "2019-03-19" ], "times": [ "19:49:16", "18:36:57", "02:45:18", "01:54:50", "18:39:06", "17:58:02", "22:04:48" ] } } } }
+{ "business_id": "--EF5N7P70J_UYBTPypYlA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-05-25", "2018-09-18", "2019-10-18" ], "times": [ "19:52:07", "16:09:44", "21:29:09" ] } } } }
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-06-07" ], "times": [ "17:54:58" ] } } } }
+{ "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-05-03", "2011-08-23", "2014-12-04", "2016-11-16" ], "times": [ "20:54:05", "20:49:45", "06:13:01", "19:25:55" ] } } } }
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-06-18", "2016-10-15" ], "times": [ "21:35:45", "18:17:51" ] } } } }
+{ "business_id": "--e8PjCNhEz32pprnPhCwQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-04-02" ], "times": [ "21:45:17" ] } } } }
+{ "business_id": "--kinfHwmtdjz03g8B8z8Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-08-27", "2015-12-19", "2018-11-27" ], "times": [ "17:49:18", "21:30:31", "15:53:50" ] } } } }
+{ "business_id": "--q6datkI-f0EoVheXNEeQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-01-28", "2014-11-16", "2015-11-15", "2015-11-15" ], "times": [ "20:56:04", "16:11:58", "19:21:53", "19:33:39" ] } } } }
+{ "business_id": "--qvQS4MigHPykD2GV0-zw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-04-11" ], "times": [ "18:30:12" ] } } } }
+{ "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-06-06", "2019-03-14" ], "times": [ "20:01:06", "22:01:52" ] } } } }
+{ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-09-29", "2018-10-20", "2018-10-20" ], "times": [ "18:55:17", "16:48:05", "22:20:24" ] } } } }
+{ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-04-23", "2014-05-04", "2014-05-11", "2014-06-04", "2015-12-05", "2017-05-15" ], "times": [ "21:11:22", "19:42:48", "19:16:08", "19:14:18", "19:22:42", "23:19:00" ] } } } }
+{ "business_id": "-0KMvRFwDWdVBeTpT11iHw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2012-07-13", "2016-12-24", "2017-08-31" ], "times": [ "21:43:57", "02:27:31", "00:35:26" ] } } } }
+{ "business_id": "-0LPtgJC31FWMrMv317p0Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2013-04-13", "2013-08-19", "2013-10-04" ], "times": [ "12:35:33", "23:35:49", "19:14:56" ] } } } }
+{ "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2016-09-10", "2018-09-08", "2019-09-13" ], "times": [ "19:26:19", "14:15:37", "22:47:25" ] } } } }
+{ "business_id": "-0RRiWDtfnS16AKCtfvBZg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27" ], "times": [ "14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20" ] } } } }
+{ "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-06-05" ], "times": [ "18:22:49" ] } } } }
+{ "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-09-24", "2014-03-10", "2015-05-27", "2015-08-29", "2018-03-16" ], "times": [ "21:37:32", "20:20:07", "00:40:24", "17:58:15", "15:03:26" ] } } } }
+{ "business_id": "-0aOudcaAyac0VJbMX-L1g", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-03-16", "2015-12-21", "2016-10-28", "2016-10-28" ], "times": [ "23:51:16", "04:48:01", "20:22:42", "20:23:00" ] } } } }
+{ "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2013-10-22", "2014-11-21" ], "times": [ "16:49:21", "17:39:24" ] } } } }
+{ "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2014-08-07", "2014-09-16", "2014-10-12", "2015-07-21", "2015-07-21" ], "times": [ "18:30:48", "20:41:45", "23:22:27", "20:43:56", "20:45:07" ] } } } }
+{ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-05-02", "2015-05-06", "2015-09-26" ], "times": [ "19:49:05", "03:52:18", "01:13:19" ] } } } }
+{ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-04-11", "2015-11-21", "2016-05-06", "2017-08-09", "2017-10-21" ], "times": [ "13:14:14", "16:05:56", "14:10:04", "15:15:10", "15:12:56" ] } } } }
+{ "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-12-03", "2016-03-17", "2016-11-02" ], "times": [ "18:44:00", "18:19:21", "15:58:38" ] } } } }
+{ "business_id": "-1E2CQu_38mkghvmZgCCRw", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-04-04" ], "times": [ "22:02:37" ] } } } }
+{ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2019-02-27" ], "times": [ "14:03:08" ] } } } }
+{ "business_id": "-23R9P2eG7VTc6DVLjFKzA", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-12-21", "2012-04-15", "2012-04-15", "2013-06-30", "2013-10-04", "2014-07-16" ], "times": [ "19:02:51", "04:21:39", "14:23:56", "22:39:51", "20:34:13", "02:28:40" ] } } } }
+{ "business_id": "-26MGfikhJiTfCI-GqmzhQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2018-06-13" ], "times": [ "20:16:07" ] } } } }
+{ "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2015-05-29", "2015-06-01" ], "times": [ "16:46:17", "15:03:53" ] } } } }
+{ "business_id": "-2hDBMaza_ldqnZdiU06LQ", "checkin_data": { "checkin_temporal": { "checkin_times": { "dates": [ "2011-10-08", "2014-08-18", "2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29", "2018-07-22", "2018-09-07", "2019-03-08" ], "times": [ "12:02:23", "02:11:11", "05:27:51", "20:15:55", "03:57:10", "01:54:42", "19:55:31", "01:42:54", "03:41:06" ] } } } }
+
diff --git a/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json b/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json
new file mode 100644
index 0000000..1611114
--- /dev/null
+++ b/asterixdb/asterix-app/data/yelp-checkin/with-composite-pk.json
@@ -0,0 +1,30 @@
+{ "checkin_id": 1, "business_id": "--1UhMGODdWsrMastO9DZw", "dates": [ "2016-04-26 19:49:16", "2016-08-30 18:36:57", "2016-10-15 02:45:18", "2016-11-18 01:54:50", "2017-04-20 18:39:06", "2017-05-03 17:58:02", "2019-03-19 22:04:48" ] }
+{ "checkin_id": 2, "business_id": "--EF5N7P70J_UYBTPypYlA", "dates": [ "2018-05-25 19:52:07", "2018-09-18 16:09:44", "2019-10-18 21:29:09" ] }
+{ "checkin_id": 3, "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg", "dates": [ "2019-06-07 17:54:58" ] }
+{ "checkin_id": 4, "business_id": "--Y1Adl1YUWfYIRSd8vkmA", "dates": [ "2011-05-03 20:54:05", "2011-08-23 20:49:45", "2014-12-04 06:13:01", "2016-11-16 19:25:55" ] }
+{ "checkin_id": 5, "business_id": "--YPwqIlRJrhHkJcjY3eiA", "dates": [ "2016-06-18 21:35:45", "2016-10-15 18:17:51" ] }
+{ "checkin_id": 6, "business_id": "--e8PjCNhEz32pprnPhCwQ", "dates": [ "2015-04-02 21:45:17" ] }
+{ "checkin_id": 7, "business_id": "--kinfHwmtdjz03g8B8z8Q", "dates": [ "2014-08-27 17:49:18", "2015-12-19 21:30:31", "2018-11-27 15:53:50" ] }
+{ "checkin_id": 8, "business_id": "--q6datkI-f0EoVheXNEeQ", "dates": [ "2014-01-28 20:56:04", "2014-11-16 16:11:58", "2015-11-15 19:21:53", "2015-11-15 19:33:39" ] }
+{ "checkin_id": 9, "business_id": "--qvQS4MigHPykD2GV0-zw", "dates": [ "2019-04-11 18:30:12" ] }
+{ "checkin_id": 10, "business_id": "--wIGbLEhlpl_UeAIyDmZQ", "dates": [ "2015-06-06 20:01:06", "2019-03-14 22:01:52" ] }
+{ "checkin_id": 11, "business_id": "-0FA-Qdi3SPYIoJz9UQw-A", "dates": [ "2018-09-29 18:55:17", "2018-10-20 16:48:05", "2018-10-20 22:20:24" ] }
+{ "checkin_id": 12, "business_id": "-0Hj1hb_XW6ybWq2M7QhGA", "dates": [ "2011-04-23 21:11:22", "2014-05-04 19:42:48", "2014-05-11 19:16:08", "2014-06-04 19:14:18", "2015-12-05 19:22:42", "2017-05-15 23:19:00" ] }
+{ "checkin_id": 13, "business_id": "-0KMvRFwDWdVBeTpT11iHw", "dates": [ "2012-07-13 21:43:57", "2016-12-24 02:27:31", "2017-08-31 00:35:26" ] }
+{ "checkin_id": 14, "business_id": "-0LPtgJC31FWMrMv317p0Q", "dates": [ "2013-04-13 12:35:33", "2013-08-19 23:35:49", "2013-10-04 19:14:56" ] }
+{ "checkin_id": 15, "business_id": "-0M3o2uWBnQZwd3hmfEwuw", "dates": [ "2016-09-10 19:26:19", "2018-09-08 14:15:37", "2019-09-13 22:47:25" ] }
+{ "checkin_id": 16, "business_id": "-0RRiWDtfnS16AKCtfvBZg", "dates": [ "2017-05-19 14:30:16", "2017-05-19 14:30:25", "2017-08-28 15:49:37", "2017-09-20 20:19:51", "2017-10-01 16:31:05", "2017-10-01 16:56:27", "2017-12-27 23:33:20" ] }
+{ "checkin_id": 17, "business_id": "-0Soj75v-XoRcf2ERr8Bmg", "dates": [ "2019-06-05 18:22:49" ] }
+{ "checkin_id": 18, "business_id": "-0ZumLlFjMh4ZW1z2nXGug", "dates": [ "2011-09-24 21:37:32", "2014-03-10 20:20:07", "2015-05-27 00:40:24", "2015-08-29 17:58:15", "2018-03-16 15:03:26" ] }
+{ "checkin_id": 19, "business_id": "-0aOudcaAyac0VJbMX-L1g", "dates": [ "2015-03-16 23:51:16", "2015-12-21 04:48:01", "2016-10-28 20:22:42", "2016-10-28 20:23:00" ] }
+{ "checkin_id": 20, "business_id": "-0b86isaXMY0v4g-V8GZ9Q", "dates": [ "2013-10-22 16:49:21", "2014-11-21 17:39:24" ] }
+{ "checkin_id": 21, "business_id": "-0d-BfFSU0bwLcnMaGRxYw", "dates": [ "2014-08-07 18:30:48", "2014-09-16 20:41:45", "2014-10-12 23:22:27", "2015-07-21 20:43:56", "2015-07-21 20:45:07" ] }
+{ "checkin_id": 22, "business_id": "-0jz6c3C6i7RG7Ag22K-Pg", "dates": [ "2015-05-02 19:49:05", "2015-05-06 03:52:18", "2015-09-26 01:13:19" ] }
+{ "checkin_id": 23, "business_id": "-0y3MZU2oYP8r1ruDP1bfQ", "dates": [ "2015-04-11 13:14:14", "2015-11-21 16:05:56", "2016-05-06 14:10:04", "2017-08-09 15:15:10", "2017-10-21 15:12:56" ] }
+{ "checkin_id": 24, "business_id": "-1BPe8UjF2_l3nVk-DFUjA", "dates": [ "2015-12-03 18:44:00", "2016-03-17 18:19:21", "2016-11-02 15:58:38" ] }
+{ "checkin_id": 25, "business_id": "-1E2CQu_38mkghvmZgCCRw", "dates": [ "2019-04-04 22:02:37" ] }
+{ "checkin_id": 26, "business_id": "-1wzk43IZ5D9Ysu6kzb5xA", "dates": [ "2019-02-27 14:03:08" ] }
+{ "checkin_id": 27, "business_id": "-23R9P2eG7VTc6DVLjFKzA", "dates": [ "2011-12-21 19:02:51", "2012-04-15 04:21:39", "2012-04-15 14:23:56", "2013-06-30 22:39:51", "2013-10-04 20:34:13", "2014-07-16 02:28:40" ] }
+{ "checkin_id": 28, "business_id": "-26MGfikhJiTfCI-GqmzhQ", "dates": [ "2018-06-13 20:16:07" ] }
+{ "checkin_id": 29, "business_id": "-2bLuJsMZ0WhI9daurVQNQ", "dates": [ "2015-05-29 16:46:17", "2015-06-01 15:03:53" ] }
+{ "checkin_id": 30, "business_id": "-2hDBMaza_ldqnZdiU06LQ", "dates": [ "2011-10-08 12:02:23", "2014-08-18 02:11:11", "2016-01-07 05:27:51", "2016-10-21 20:15:55", "2016-12-01 03:57:10", "2016-12-29 01:54:42", "2018-07-22 19:55:31", "2018-09-07 01:42:54", "2019-03-08 03:41:06" ] }
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 9131202..eebb24e 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-app</artifactId>
<licenses>
@@ -154,7 +154,6 @@
</ignoredUsedUndeclaredDependencies>
<usedDependencies combine.children="append">
<usedDependency>org.apache.hadoop:hadoop-common</usedDependency>
- <usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
<usedDependency>org.apache.asterix:asterix-geo</usedDependency>
</usedDependencies>
<ignoredUnusedDeclaredDependencies>
@@ -756,12 +755,6 @@
</dependency>
<dependency>
<groupId>org.apache.asterix</groupId>
- <artifactId>asterix-fuzzyjoin</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
<artifactId>asterix-geo</artifactId>
<version>${project.version}</version>
<scope>test</scope>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 920b0d9..326ddcc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -139,7 +139,7 @@
CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, CompilerProperties.COMPILER_INDEXONLY_KEY,
CompilerProperties.COMPILER_INTERNAL_SANITYCHECK_KEY,
CompilerProperties.COMPILER_EXTERNAL_FIELD_PUSHDOWN_KEY, CompilerProperties.COMPILER_SUBPLAN_MERGE_KEY,
- CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY,
+ CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
index 1168567..4cc6e43 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/io/PersistedResourceRegistry.java
@@ -111,6 +111,8 @@
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.impls.PrefixMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResource;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.StopwordsFullTextFilterEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
@@ -273,6 +275,10 @@
registeredClasses.put("DelimitedUTF8StringBinaryTokenizerFactory",
DelimitedUTF8StringBinaryTokenizerFactory.class);
+ // IFullTextConfigFactory
+ registeredClasses.put("FullTextConfigEvaluatorFactory", FullTextConfigEvaluatorFactory.class);
+ registeredClasses.put("StopwordsFullTextFilterEvaluatorFactory", StopwordsFullTextFilterEvaluatorFactory.class);
+
// ITokenFactory
registeredClasses.put("AListElementTokenFactory", AListElementTokenFactory.class);
registeredClasses.put("HashedUTF8NGramTokenFactory", HashedUTF8NGramTokenFactory.class);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 4b6f01d..1d1466c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -21,6 +21,7 @@
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import static org.apache.asterix.common.utils.IdentifierUtil.dataverse;
+import static org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement.FIELD_TYPE_STOPWORDS;
import java.io.File;
import java.io.FileInputStream;
@@ -41,6 +42,7 @@
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
import org.apache.asterix.active.ActivityState;
import org.apache.asterix.active.EntityId;
@@ -113,6 +115,8 @@
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -126,6 +130,8 @@
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -164,6 +170,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -186,6 +194,10 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeSignature;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.asterix.runtime.fulltext.IFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
import org.apache.asterix.translator.AbstractLangTranslator;
import org.apache.asterix.translator.ClientRequest;
@@ -236,10 +248,14 @@
import org.apache.hyracks.control.common.controllers.CCConfig;
import org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+
/*
* Provides functionality for executing a batch of Query statements (queries included)
* sequentially.
@@ -346,6 +362,12 @@
case CREATE_INDEX:
handleCreateIndexStatement(metadataProvider, stmt, hcc, requestParameters);
break;
+ case CREATE_FULL_TEXT_FILTER:
+ handleCreateFullTextFilterStatement(metadataProvider, stmt);
+ break;
+ case CREATE_FULL_TEXT_CONFIG:
+ handleCreateFullTextConfigStatement(metadataProvider, stmt);
+ break;
case TYPE_DECL:
handleCreateTypeStatement(metadataProvider, stmt);
break;
@@ -361,6 +383,12 @@
case INDEX_DROP:
handleIndexDropStatement(metadataProvider, stmt, hcc, requestParameters);
break;
+ case FULL_TEXT_FILTER_DROP:
+ handleFullTextFilterDrop(metadataProvider, stmt, hcc, requestParameters);
+ break;
+ case FULL_TEXT_CONFIG_DROP:
+ handleFullTextConfigDrop(metadataProvider, stmt, hcc, requestParameters);
+ break;
case TYPE_DROP:
handleTypeDropStatement(metadataProvider, stmt);
break;
@@ -450,9 +478,7 @@
handleExternalDatasetRefreshStatement(metadataProvider, stmt, hcc);
break;
case WRITE:
- Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(stmt);
- writerFactory = (result.first != null) ? result.first : writerFactory;
- outputFile = result.second;
+ //Deprecated.
break;
case FUNCTION_DECL:
handleDeclareFunctionStatement(metadataProvider, stmt);
@@ -986,10 +1012,12 @@
CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
String datasetName = stmtCreateIndex.getDatasetName().getValue();
String indexName = stmtCreateIndex.getIndexName().getValue();
+ String fullTextConfigName = stmtCreateIndex.getFullTextConfigName();
metadataProvider.validateDatabaseObjectName(stmtCreateIndex.getDataverseName(), indexName,
stmt.getSourceLocation());
DataverseName dataverseName = getActiveDataverseName(stmtCreateIndex.getDataverseName());
- lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName);
+ lockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName, datasetName,
+ fullTextConfigName);
try {
doCreateIndex(metadataProvider, stmtCreateIndex, dataverseName, datasetName, hcc, requestParameters);
} finally {
@@ -1019,7 +1047,9 @@
DatasetType datasetType = ds.getDatasetType();
IndexType indexType = stmtCreateIndex.getIndexType();
- boolean isSecondaryPrimary = stmtCreateIndex.getFieldExprs().isEmpty();
+ List<CreateIndexStatement.IndexedElement> indexedElements = stmtCreateIndex.getIndexedElements();
+ int indexedElementsCount = indexedElements.size();
+ boolean isSecondaryPrimary = indexedElementsCount == 0;
validateIndexType(datasetType, indexType, isSecondaryPrimary, sourceLoc);
String indexName = stmtCreateIndex.getIndexName().getValue();
@@ -1034,109 +1064,233 @@
}
}
- List<Integer> keySourceIndicators;
- if (isSecondaryPrimary && datasetType == DatasetType.INTERNAL) {
- // find keySourceIndicators for secondary primary index since the parser isn't aware of them
- keySourceIndicators = ((InternalDatasetDetails) ds.getDatasetDetails()).getKeySourceIndicator();
- } else {
- keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
- }
- // disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
- if (keySourceIndicators.stream().anyMatch(fieldSource -> fieldSource == 1) && !isSecondaryPrimary) {
- throw new AsterixException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Cannot create index on meta fields");
- }
Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
ds.getItemTypeDataverseName(), ds.getItemTypeName());
ARecordType aRecordType = (ARecordType) dt.getDatatype();
+ /* TODO: unused for now becase indexes on meta are disabled -- see below
ARecordType metaRecordType = null;
if (ds.hasMetaPart()) {
Datatype metaDt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
ds.getMetaItemTypeDataverseName(), ds.getMetaItemTypeName());
metaRecordType = (ARecordType) metaDt.getDatatype();
}
+ */
- List<List<String>> indexFields = new ArrayList<>();
- List<IAType> indexFieldTypes = new ArrayList<>();
- int keyIndex = 0;
+ List<List<IAType>> indexFieldTypes = new ArrayList<>(indexedElementsCount);
+ boolean hadUnnest = false;
boolean overridesFieldTypes = false;
// this set is used to detect duplicates in the specified keys in the create
// index statement
// e.g. CREATE INDEX someIdx on dataset(id,id).
- // checking only the names is not enough. Need also to check the source
- // indicators for cases like:
- // CREATE INDEX someIdx on dataset(meta().id, id)
- Set<Pair<List<String>, Integer>> indexKeysSet = new HashSet<>();
+ // checking only the names is not enough.
+ // Need also to check the source indicators for the most general case
+ // (even though indexes on meta fields are curently disabled -- see below)
+ Set<Triple<Integer, List<List<String>>, List<List<String>>>> indexKeysSet = new HashSet<>();
- for (Pair<List<String>, IndexedTypeExpression> fieldExpr : stmtCreateIndex.getFieldExprs()) {
- IAType fieldType = null;
- ARecordType subType =
- KeyFieldTypeUtil.chooseSource(keySourceIndicators, keyIndex, aRecordType, metaRecordType);
- boolean isOpen = subType.isOpen();
- int i = 0;
- if (fieldExpr.first.size() > 1 && !isOpen) {
- while (i < fieldExpr.first.size() - 1 && !isOpen) {
- subType = (ARecordType) subType.getFieldType(fieldExpr.first.get(i));
- i++;
- isOpen = subType.isOpen();
- }
+ for (CreateIndexStatement.IndexedElement indexedElement : indexedElements) {
+ // disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
+ if (indexedElement.getSourceIndicator() != Index.RECORD_INDICATOR) {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+ "Cannot create index on meta fields");
}
- if (fieldExpr.second == null) {
- fieldType = subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size()));
+ ARecordType sourceRecordType = aRecordType;
+ IAType inputTypePrime;
+ boolean inputTypeNullable, inputTypeMissable;
+ List<Pair<List<String>, IndexedTypeExpression>> projectList = indexedElement.getProjectList();
+ int projectCount = projectList.size();
+ if (indexedElement.hasUnnest()) {
+ if (indexType != IndexType.ARRAY) {
+ throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE,
+ indexedElement.getSourceLocation(), String.valueOf(indexType));
+ }
+ // allow only 1 unnesting element in ARRAY index
+ if (hadUnnest) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+ "Cannot create composite index with multiple array fields using different arrays");
+ }
+ hadUnnest = true;
+ if (projectCount == 0) {
+ // Note. UNNEST with no SELECT is supposed to have 1 project element with 'null' path
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+ "Invalid index element");
+ }
+ Triple<IAType, Boolean, Boolean> unnestTypeResult = KeyFieldTypeUtil.getKeyUnnestType(
+ sourceRecordType, indexedElement.getUnnestList(), indexedElement.getSourceLocation());
+ if (unnestTypeResult == null) {
+ inputTypePrime = null; // = ANY
+ inputTypeNullable = inputTypeMissable = true;
+ } else {
+ inputTypePrime = unnestTypeResult.first;
+ inputTypeNullable = unnestTypeResult.second;
+ inputTypeMissable = unnestTypeResult.third;
+ }
} else {
- if (!stmtCreateIndex.isEnforced() && indexType != IndexType.BTREE) {
- throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc, indexType);
+ if (projectCount != 1) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, indexedElement.getSourceLocation(),
+ "Invalid index element");
}
- if (stmtCreateIndex.isEnforced() && !fieldExpr.second.isUnknownable()) {
- throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL, sourceLoc,
- String.valueOf(fieldExpr.first));
- }
- // don't allow creating an enforced index on a closed-type field, fields that
- // are part of schema.
- // get the field type, if it's not null, then the field is closed-type
- if (stmtCreateIndex.isEnforced()
- && subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size())) != null) {
- throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD, sourceLoc,
- String.valueOf(fieldExpr.first));
- }
- if (!isOpen) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Typed index on \""
- + fieldExpr.first + "\" field could be created only for open datatype");
- }
- if (stmtCreateIndex.hasMetaField()) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Typed open index can only be created on the record part");
- }
- Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(dataverseName, indexName,
- fieldExpr.second.getType(), dataverseName, mdTxnCtx);
- TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
- fieldType = typeMap.get(typeSignature);
- overridesFieldTypes = true;
- }
- if (fieldType == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, fieldExpr.second == null
- ? String.valueOf(fieldExpr.first) : String.valueOf(fieldExpr.second));
+ inputTypePrime = sourceRecordType;
+ inputTypeNullable = inputTypeMissable = false;
}
- // try to add the key & its source to the set of keys, if key couldn't be added,
- // there is a duplicate
- if (!indexKeysSet
- .add(new Pair<>(fieldExpr.first, stmtCreateIndex.getFieldSourceIndicators().get(keyIndex)))) {
- throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD, sourceLoc,
- String.valueOf(fieldExpr.first));
+ // at this point 'inputTypePrime' is either a record, or if we had unnest then it could also be anything else.
+ List<IAType> fieldTypes = new ArrayList<>(projectCount);
+ for (int i = 0; i < projectCount; i++) {
+ Pair<List<String>, IndexedTypeExpression> projectPair = projectList.get(i);
+ List<String> projectPath = projectPair.first;
+ IndexedTypeExpression projectTypeExpr = projectPair.second;
+ IAType projectTypePrime;
+ boolean projectTypeNullable, projectTypeMissable;
+ if (projectPath == null) {
+ boolean emptyPathOk = indexedElement.hasUnnest() && i == 0;
+ if (!emptyPathOk) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ indexedElement.getSourceLocation(), "Invalid index element");
+ }
+ projectTypePrime = inputTypePrime;
+ projectTypeNullable = inputTypeNullable;
+ projectTypeMissable = inputTypeMissable;
+ } else if (inputTypePrime == null) {
+ projectTypePrime = null; // ANY
+ projectTypeNullable = projectTypeMissable = true;
+ } else {
+ if (inputTypePrime.getTypeTag() != ATypeTag.OBJECT) {
+ throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+ inputTypePrime.getTypeTag());
+ }
+ ARecordType inputTypePrimeRecord = (ARecordType) inputTypePrime;
+ Triple<IAType, Boolean, Boolean> projectTypeResult = KeyFieldTypeUtil.getKeyProjectType(
+ inputTypePrimeRecord, projectPath, indexedElement.getSourceLocation());
+ if (projectTypeResult != null) {
+ projectTypePrime = projectTypeResult.first;
+ projectTypeNullable = inputTypeNullable || projectTypeResult.second;
+ projectTypeMissable = inputTypeMissable || projectTypeResult.third;
+ } else {
+ projectTypePrime = null; // ANY
+ projectTypeNullable = projectTypeMissable = true;
+ }
+ }
+
+ IAType fieldTypePrime;
+ boolean fieldTypeNullable, fieldTypeMissable;
+ if (projectTypeExpr == null) {
+ fieldTypePrime = projectTypePrime;
+ fieldTypeNullable = projectTypeNullable;
+ fieldTypeMissable = projectTypeMissable;
+ } else {
+ if (stmtCreateIndex.isEnforced()) {
+ if (!projectTypeExpr.isUnknownable()) {
+ throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL,
+ indexedElement.getSourceLocation(), String.valueOf(projectPath));
+ }
+ // don't allow creating an enforced index on a closed-type field, fields that
+ // are part of schema get the field type, if it's not null, then the field is closed-type
+ if (projectTypePrime != null) {
+ throw new CompilationException(ErrorCode.INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD,
+ indexedElement.getSourceLocation(), String.valueOf(projectPath));
+ }
+ } else {
+ if (indexType != IndexType.BTREE && indexType != IndexType.ARRAY) {
+ throw new CompilationException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED,
+ indexedElement.getSourceLocation(), indexType);
+ }
+ if (projectTypePrime != null) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ indexedElement.getSourceLocation(), "Typed index on \"" + projectPath
+ + "\" field could be created only for open datatype");
+ }
+ }
+
+ Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(dataverseName, indexName,
+ projectTypeExpr.getType(), dataverseName, mdTxnCtx);
+ TypeSignature typeSignature = new TypeSignature(dataverseName, indexName);
+ fieldTypePrime = typeMap.get(typeSignature);
+ // BACK-COMPAT: keep prime type only if we're overriding field types
+ fieldTypeNullable = fieldTypeMissable = false;
+ overridesFieldTypes = true;
+ }
+
+ if (fieldTypePrime == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_TYPE, indexedElement.getSourceLocation(),
+ String.valueOf(projectPath));
+ }
+ validateIndexFieldType(indexType, fieldTypePrime, projectPath, indexedElement.getSourceLocation());
+
+ IAType fieldType =
+ KeyFieldTypeUtil.makeUnknownableType(fieldTypePrime, fieldTypeNullable, fieldTypeMissable);
+ fieldTypes.add(fieldType);
}
- indexFields.add(fieldExpr.first);
- indexFieldTypes.add(fieldType);
- ++keyIndex;
+ // Try to add the key & its source to the set of keys for duplicate detection.
+ if (!indexKeysSet.add(indexedElement.toIdentifier())) {
+ throw new AsterixException(ErrorCode.INDEX_ILLEGAL_REPETITIVE_FIELD,
+ indexedElement.getSourceLocation(), indexedElement.getProjectListDisplayForm());
+ }
+
+ indexFieldTypes.add(fieldTypes);
}
- validateIndexKeyFields(stmtCreateIndex, keySourceIndicators, aRecordType, metaRecordType, indexFields,
- indexFieldTypes);
+ Index.IIndexDetails indexDetails;
+ if (Index.IndexCategory.of(indexType) == Index.IndexCategory.ARRAY) {
+ if (!hadUnnest) {
+ // prohibited by the grammar
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(indexType));
+ }
+ if (stmtCreateIndex.isEnforced()) {
+ // not supported yet.
+ throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+ String.valueOf(indexType));
+ }
+ if (indexedElementsCount > 1) {
+ // TODO (GLENN): Add in support for composite atomic / array indexes.
+ throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+ String.valueOf(indexType));
+ }
- Index newIndex = new Index(dataverseName, datasetName, indexName, indexType, indexFields,
- keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(), overridesFieldTypes,
+ List<Index.ArrayIndexElement> indexElementList = new ArrayList<>(indexedElementsCount);
+ for (int i = 0; i < indexedElementsCount; i++) {
+ CreateIndexStatement.IndexedElement indexedElement = indexedElements.get(i);
+ List<List<String>> projectList =
+ indexedElement.getProjectList().stream().map(Pair::getFirst).collect(Collectors.toList());
+ indexElementList.add(new Index.ArrayIndexElement(indexedElement.getUnnestList(), projectList,
+ indexFieldTypes.get(i), indexedElement.getSourceIndicator()));
+ }
+ indexDetails = new Index.ArrayIndexDetails(indexElementList, overridesFieldTypes);
+ } else {
+ List<List<String>> keyFieldNames = new ArrayList<>(indexedElementsCount);
+ List<IAType> keyFieldTypes = new ArrayList<>(indexedElementsCount);
+ List<Integer> keyFieldSourceIndicators = new ArrayList<>(indexedElementsCount);
+ if (isSecondaryPrimary) {
+ // BACK-COMPAT: secondary primary index has one source indicator
+ // which is set to META_RECORD_INDICATOR
+ keyFieldSourceIndicators.add(Index.META_RECORD_INDICATOR);
+ } else {
+ for (int i = 0; i < indexedElementsCount; i++) {
+ CreateIndexStatement.IndexedElement indexedElement = indexedElements.get(i);
+ keyFieldNames.add(indexedElement.getProjectList().get(0).first);
+ keyFieldTypes.add(indexFieldTypes.get(i).get(0));
+ keyFieldSourceIndicators.add(indexedElement.getSourceIndicator());
+ }
+ }
+ switch (Index.IndexCategory.of(indexType)) {
+ case VALUE:
+ indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators,
+ keyFieldTypes, overridesFieldTypes);
+ break;
+ case TEXT:
+ indexDetails = new Index.TextIndexDetails(keyFieldNames, keyFieldSourceIndicators,
+ keyFieldTypes, overridesFieldTypes, stmtCreateIndex.getGramLength(),
+ stmtCreateIndex.getFullTextConfigName());
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(indexType));
+ }
+ }
+
+ Index newIndex = new Index(dataverseName, datasetName, indexName, indexType, indexDetails,
stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
bActiveTxn = false; // doCreateIndexImpl() takes over the current transaction
@@ -1150,6 +1304,135 @@
}
}
+ public void handleCreateFullTextFilterStatement(MetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
+ CreateFullTextFilterStatement stmtCreateFilter = (CreateFullTextFilterStatement) stmt;
+ String fullTextFilterName = stmtCreateFilter.getFilterName();
+ metadataProvider.validateDatabaseObjectName(stmtCreateFilter.getDataverseName(), fullTextFilterName,
+ stmt.getSourceLocation());
+ DataverseName dataverseName = getActiveDataverseName(stmtCreateFilter.getDataverseName());
+
+ lockUtil.createFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
+ try {
+ doCreateFullTextFilter(metadataProvider, stmtCreateFilter, dataverseName);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ protected void doCreateFullTextFilter(MetadataProvider metadataProvider,
+ CreateFullTextFilterStatement stmtCreateFilter, DataverseName dataverseName) throws Exception {
+ AbstractFullTextFilterDescriptor filterDescriptor;
+
+ String filterType = stmtCreateFilter.getFilterType();
+ if (filterType == null) {
+ throw new CompilationException(ErrorCode.PARSE_ERROR, stmtCreateFilter.getSourceLocation(),
+ "full-text filter type is null");
+ }
+
+ switch (filterType) {
+ case FIELD_TYPE_STOPWORDS: {
+ filterDescriptor = new StopwordsFullTextFilterDescriptor(dataverseName,
+ stmtCreateFilter.getFilterName(), stmtCreateFilter.getStopwordsList());
+ break;
+ }
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, stmtCreateFilter.getSourceLocation(),
+ "Unexpected full-text filter type: " + filterType);
+ }
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, stmtCreateFilter.getSourceLocation(),
+ dataverseName);
+ }
+
+ String filterName = stmtCreateFilter.getFilterName();
+ FullTextFilterMetadataEntity existingFilter =
+ MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, filterName);
+ if (existingFilter != null) {
+ if (stmtCreateFilter.getIfNotExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_ALREADY_EXISTS,
+ stmtCreateFilter.getSourceLocation(), filterName);
+ }
+ }
+
+ MetadataManager.INSTANCE.addFullTextFilter(mdTxnCtx, new FullTextFilterMetadataEntity(filterDescriptor));
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
+ public void handleCreateFullTextConfigStatement(MetadataProvider metadataProvider, Statement stmt)
+ throws Exception {
+ CreateFullTextConfigStatement stmtCreateConfig = (CreateFullTextConfigStatement) stmt;
+ String configName = stmtCreateConfig.getConfigName();
+ metadataProvider.validateDatabaseObjectName(stmtCreateConfig.getDataverseName(), configName,
+ stmt.getSourceLocation());
+ DataverseName dataverseName = getActiveDataverseName(stmtCreateConfig.getDataverseName());
+ ImmutableList<String> filterNames = stmtCreateConfig.getFilterNames();
+
+ lockUtil.createFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName,
+ filterNames);
+ try {
+ doCreateFullTextConfig(metadataProvider, stmtCreateConfig, dataverseName, configName, filterNames);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ protected void doCreateFullTextConfig(MetadataProvider metadataProvider,
+ CreateFullTextConfigStatement stmtCreateConfig, DataverseName dataverseName, String configName,
+ ImmutableList<String> filterNames) throws Exception {
+
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+ try {
+ FullTextConfigMetadataEntity existingConfig =
+ MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, configName);
+ if (existingConfig != null) {
+ if (stmtCreateConfig.getIfNotExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.FULL_TEXT_CONFIG_ALREADY_EXISTS,
+ stmtCreateConfig.getSourceLocation(), configName);
+ }
+ }
+
+ ImmutableList.Builder<IFullTextFilterDescriptor> filterDescriptorsBuilder =
+ ImmutableList.<IFullTextFilterDescriptor> builder();
+ for (String filterName : filterNames) {
+ FullTextFilterMetadataEntity filterMetadataEntity =
+ MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, filterName);
+ if (filterMetadataEntity == null) {
+ throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_NOT_FOUND,
+ stmtCreateConfig.getSourceLocation(), filterName);
+ }
+ }
+
+ TokenizerCategory tokenizerCategory = stmtCreateConfig.getTokenizerCategory();
+ FullTextConfigDescriptor configDescriptor =
+ new FullTextConfigDescriptor(dataverseName, configName, tokenizerCategory, filterNames);
+ FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(configDescriptor);
+
+ MetadataManager.INSTANCE.addFullTextConfig(mdTxnCtx, configMetadataEntity);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
private void doCreateIndexImpl(IHyracksClientConnection hcc, MetadataProvider metadataProvider, Dataset ds,
Index index, EnumSet<JobFlag> jobFlags, SourceLocation sourceLoc) throws Exception {
ProgressState progress = ProgressState.NO_PROGRESS;
@@ -1204,9 +1487,9 @@
// Add an entry for the files index
filesIndex = new Index(index.getDataverseName(), index.getDatasetName(),
IndexingConstants.getFilesIndexName(index.getDatasetName()), IndexType.BTREE,
- ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
- ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false, false, false,
- MetadataUtil.PENDING_ADD_OP);
+ new Index.ValueIndexDetails(ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES, null,
+ ExternalIndexingOperations.FILE_INDEX_FIELD_TYPES, false),
+ false, false, MetadataUtil.PENDING_ADD_OP);
MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), filesIndex);
// Add files to the external files index
for (ExternalFile file : externalFilesSnapshot) {
@@ -1215,10 +1498,6 @@
// This is the first index for the external dataset, replicate the files index
spec = ExternalIndexingOperations.buildFilesIndexCreateJobSpec(ds, externalFilesSnapshot,
metadataProvider);
- if (spec == null) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Failed to create job spec for replicating files index for external " + dataset());
- }
filesIndexReplicated = true;
runJob(hcc, spec, jobFlags);
}
@@ -1226,16 +1505,54 @@
// check whether there exists another enforced index on the same field
if (index.isEnforced()) {
+ List<List<String>> indexKeyFieldNames;
+ List<IAType> indexKeyFieldTypes;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ indexKeyFieldNames = valueIndexDetails.getKeyFieldNames();
+ indexKeyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ indexKeyFieldNames = textIndexDetails.getKeyFieldNames();
+ indexKeyFieldTypes = textIndexDetails.getKeyFieldTypes();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+ }
List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(
metadataProvider.getMetadataTxnContext(), index.getDataverseName(), index.getDatasetName());
for (Index existingIndex : indexes) {
- if (existingIndex.getKeyFieldNames().equals(index.getKeyFieldNames())
- && !existingIndex.getKeyFieldTypes().equals(index.getKeyFieldTypes())
- && existingIndex.isEnforced()) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc, "Cannot create index "
- + index.getIndexName() + " , enforced index " + existingIndex.getIndexName()
- + " on field \"" + StringUtils.join(index.getKeyFieldNames(), ',')
- + "\" is already defined with type \"" + existingIndex.getKeyFieldTypes() + "\"");
+ if (!existingIndex.isEnforced()) {
+ continue;
+ }
+ List<List<String>> existingIndexKeyFieldNames;
+ List<IAType> existingIndexKeyFieldTypes;
+ switch (Index.IndexCategory.of(existingIndex.getIndexType())) {
+ case VALUE:
+ Index.ValueIndexDetails valueIndexDetails =
+ (Index.ValueIndexDetails) existingIndex.getIndexDetails();
+ existingIndexKeyFieldNames = valueIndexDetails.getKeyFieldNames();
+ existingIndexKeyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+ break;
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails =
+ (Index.TextIndexDetails) existingIndex.getIndexDetails();
+ existingIndexKeyFieldNames = textIndexDetails.getKeyFieldNames();
+ existingIndexKeyFieldTypes = textIndexDetails.getKeyFieldTypes();
+ break;
+ default:
+ // ARRAY indexed cannot be enforced yet.
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "");
+ }
+ if (existingIndexKeyFieldNames.equals(indexKeyFieldNames)
+ && !existingIndexKeyFieldTypes.equals(indexKeyFieldTypes)) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ "Cannot create index " + index.getIndexName() + " , enforced index "
+ + existingIndex.getIndexName() + " on field \""
+ + StringUtils.join(indexKeyFieldNames, ',')
+ + "\" is already defined with type \"" + existingIndexKeyFieldTypes + "\"");
}
}
}
@@ -1400,13 +1717,15 @@
if (datasetType == DatasetType.EXTERNAL && isSecondaryPrimaryIndex) {
throw new CompilationException(ErrorCode.CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET);
}
+ if (indexType != IndexType.BTREE && isSecondaryPrimaryIndex) {
+ throw new CompilationException(ErrorCode.COMPILATION_INCOMPATIBLE_INDEX_TYPE, sourceLoc,
+ String.valueOf(indexType));
+ }
}
- protected void validateIndexKeyFields(CreateIndexStatement stmtCreateIndex, List<Integer> keySourceIndicators,
- ARecordType aRecordType, ARecordType metaRecordType, List<List<String>> indexFields,
- List<IAType> indexFieldTypes) throws AlgebricksException {
- ValidateUtil.validateKeyFields(aRecordType, metaRecordType, indexFields, keySourceIndicators, indexFieldTypes,
- stmtCreateIndex.getIndexType(), stmtCreateIndex.getSourceLocation());
+ protected void validateIndexFieldType(IndexType indexType, IAType fieldType, List<String> displayFieldName,
+ SourceLocation sourceLoc) throws AlgebricksException {
+ ValidateUtil.validateIndexFieldType(indexType, fieldType, displayFieldName, sourceLoc);
}
protected void handleCreateTypeStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
@@ -1806,10 +2125,8 @@
// #. mark PendingDropOp on the existing index
MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
- index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
- index.isOverridingKeyFieldTypes(), index.isEnforced(), index.isPrimaryIndex(),
- MetadataUtil.PENDING_DROP_OP));
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+ index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1859,10 +2176,9 @@
externalIndex.getIndexName());
MetadataManager.INSTANCE.addIndex(mdTxnCtx,
new Index(dataverseName, datasetName, externalIndex.getIndexName(),
- externalIndex.getIndexType(), externalIndex.getKeyFieldNames(),
- externalIndex.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
- index.isOverridingKeyFieldTypes(), index.isEnforced(),
- externalIndex.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
+ externalIndex.getIndexType(), externalIndex.getIndexDetails(),
+ externalIndex.isEnforced(), externalIndex.isPrimaryIndex(),
+ MetadataUtil.PENDING_DROP_OP));
}
}
}
@@ -1870,10 +2186,8 @@
// #. mark PendingDropOp on the existing index
MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
MetadataManager.INSTANCE.addIndex(mdTxnCtx,
- new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getKeyFieldNames(),
- index.getKeyFieldSourceIndicators(), index.getKeyFieldTypes(),
- index.isOverridingKeyFieldTypes(), index.isEnforced(), index.isPrimaryIndex(),
- MetadataUtil.PENDING_DROP_OP));
+ new Index(dataverseName, datasetName, indexName, index.getIndexType(), index.getIndexDetails(),
+ index.isEnforced(), index.isPrimaryIndex(), MetadataUtil.PENDING_DROP_OP));
// #. commit the existing transaction before calling runJob.
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -1941,6 +2255,95 @@
}
}
+ protected void handleFullTextFilterDrop(MetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
+ FullTextFilterDropStatement stmtFilterDrop = (FullTextFilterDropStatement) stmt;
+ DataverseName dataverseName = getActiveDataverseName(stmtFilterDrop.getDataverseName());
+ String fullTextFilterName = stmtFilterDrop.getFilterName();
+
+ lockUtil.dropFullTextFilterBegin(lockManager, metadataProvider.getLocks(), dataverseName, fullTextFilterName);
+ try {
+ doDropFullTextFilter(metadataProvider, stmtFilterDrop, dataverseName, fullTextFilterName);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ protected void doDropFullTextFilter(MetadataProvider metadataProvider, FullTextFilterDropStatement stmtFilterDrop,
+ DataverseName dataverseName, String fullTextFilterName) throws AlgebricksException, RemoteException {
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ try {
+ FullTextFilterMetadataEntity filter =
+ MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, fullTextFilterName);
+ if (filter == null) {
+ if (stmtFilterDrop.getIfExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.FULL_TEXT_FILTER_NOT_FOUND,
+ stmtFilterDrop.getSourceLocation(), fullTextFilterName);
+ }
+ }
+
+ MetadataManager.INSTANCE.dropFullTextFilter(mdTxnCtx, dataverseName, fullTextFilterName);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
+ protected void handleFullTextConfigDrop(MetadataProvider metadataProvider, Statement stmt,
+ IHyracksClientConnection hcc, IRequestParameters requestParameters)
+ throws AlgebricksException, RemoteException {
+ FullTextConfigDropStatement stmtConfigDrop = (FullTextConfigDropStatement) stmt;
+ DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
+ String configName = stmtConfigDrop.getConfigName();
+
+ lockUtil.dropFullTextConfigBegin(lockManager, metadataProvider.getLocks(), dataverseName, configName);
+ try {
+ doDropFullTextConfig(metadataProvider, stmtConfigDrop, hcc, requestParameters);
+ } finally {
+ metadataProvider.getLocks().unlock();
+ }
+ }
+
+ private void doDropFullTextConfig(MetadataProvider metadataProvider, FullTextConfigDropStatement stmtConfigDrop,
+ IHyracksClientConnection hcc, IRequestParameters requestParameters)
+ throws RemoteException, AlgebricksException {
+ // If the config name is null, then it means the default config
+ if (Strings.isNullOrEmpty(stmtConfigDrop.getConfigName())) {
+ throw new CompilationException(ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED,
+ stmtConfigDrop.getSourceLocation());
+ }
+
+ DataverseName dataverseName = getActiveDataverseName(stmtConfigDrop.getDataverseName());
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ metadataProvider.setMetadataTxnContext(mdTxnCtx);
+ String fullTextConfigName = stmtConfigDrop.getConfigName();
+
+ try {
+ FullTextConfigMetadataEntity configMetadataEntity =
+ MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, fullTextConfigName);
+ if (configMetadataEntity == null) {
+ if (stmtConfigDrop.getIfExists()) {
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.FULL_TEXT_CONFIG_NOT_FOUND,
+ stmtConfigDrop.getSourceLocation(), fullTextConfigName);
+ }
+ }
+
+ MetadataManager.INSTANCE.dropFullTextConfig(mdTxnCtx, dataverseName, fullTextConfigName);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ } catch (Exception e) {
+ abort(e, e, mdTxnCtx);
+ throw e;
+ }
+ }
+
protected void handleTypeDropStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
SourceLocation sourceLoc = stmtTypeDrop.getSourceLocation();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
index 1040781..c668fb6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
@@ -201,7 +201,8 @@
storageComponentProvider.getStorageManager(), secondaryIndexInfo.fileSplitProvider);
IIndexDataflowHelperFactory primaryIndexHelperFactory = new IndexDataflowHelperFactory(
storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
- int[] fieldPermutation = new int[secondaryIndex.getKeyFieldNames().size()];
+ Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+ int[] fieldPermutation = new int[secondaryIndexDetails.getKeyFieldNames().size()];
for (int i = 0; i < fieldPermutation.length; i++) {
fieldPermutation[i] = i;
}
@@ -257,8 +258,10 @@
// for the index, we will have to create an assign operator that extract the sk
// then the secondary LSMInsertDeleteOperatorNodePushable
if (secondaryIndex != null) {
- List<List<String>> skNames = secondaryIndex.getKeyFieldNames();
- List<Integer> indicators = secondaryIndex.getKeyFieldSourceIndicators();
+ Index.ValueIndexDetails secondaryIndexDetails =
+ (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+ List<List<String>> skNames = secondaryIndexDetails.getKeyFieldNames();
+ List<Integer> indicators = secondaryIndexDetails.getKeyFieldSourceIndicators();
IScalarEvaluatorFactory[] secondaryFieldAccessEvalFactories =
new IScalarEvaluatorFactory[skNames.size()];
for (int i = 0; i < skNames.size(); i++) {
@@ -266,14 +269,15 @@
? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
: recordType;
int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
- secondaryFieldAccessEvalFactories[i] =
- mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
- sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
+ secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ mdProvider.getFunctionManager(), sourceType, skNames.get(i), pos, null);
}
// outColumns are computed inside the assign runtime
int[] outColumns = new int[skNames.size()];
// projection list include old and new (primary and secondary keys)
- int[] projectionList = new int[skNames.size() + primaryIndexInfo.index.getKeyFieldNames().size()];
+ Index.ValueIndexDetails primaryIndexDetails =
+ (Index.ValueIndexDetails) primaryIndexInfo.index.getIndexDetails();
+ int[] projectionList = new int[skNames.size() + primaryIndexDetails.getKeyFieldNames().size()];
for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
outColumns[i] = primaryIndexInfo.rDesc.getFieldCount() + i;
}
@@ -281,7 +285,7 @@
for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
projectionList[projCount++] = primaryIndexInfo.rDesc.getFieldCount() + i;
}
- for (int i = 0; i < primaryIndexInfo.index.getKeyFieldNames().size(); i++) {
+ for (int i = 0; i < primaryIndexDetails.getKeyFieldNames().size(); i++) {
projectionList[projCount++] = i;
}
IPushRuntime assignOp =
@@ -354,8 +358,10 @@
// for the index, we will have to create an assign operator that extract the sk
// then the secondary LSMInsertDeleteOperatorNodePushable
if (secondaryIndex != null) {
- List<List<String>> skNames = secondaryIndex.getKeyFieldNames();
- List<Integer> indicators = secondaryIndex.getKeyFieldSourceIndicators();
+ Index.ValueIndexDetails secondaryIndexDetails =
+ (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+ List<List<String>> skNames = secondaryIndexDetails.getKeyFieldNames();
+ List<Integer> indicators = secondaryIndexDetails.getKeyFieldSourceIndicators();
IScalarEvaluatorFactory[] secondaryFieldAccessEvalFactories =
new IScalarEvaluatorFactory[skNames.size()];
for (int i = 0; i < skNames.size(); i++) {
@@ -363,14 +369,15 @@
? indicators.get(i).intValue() == Index.RECORD_INDICATOR ? recordType : metaType
: recordType;
int pos = skNames.get(i).size() > 1 ? -1 : sourceType.getFieldIndex(skNames.get(i).get(0));
- secondaryFieldAccessEvalFactories[i] =
- mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(mdProvider.getFunctionManager(),
- sourceType, secondaryIndex.getKeyFieldNames().get(i), pos, null);
+ secondaryFieldAccessEvalFactories[i] = mdProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ mdProvider.getFunctionManager(), sourceType, skNames.get(i), pos, null);
}
// outColumns are computed inside the assign runtime
int[] outColumns = new int[skNames.size()];
// projection list include old and new (primary and secondary keys)
- int[] projectionList = new int[skNames.size() + primaryIndexInfo.index.getKeyFieldNames().size()];
+ Index.ValueIndexDetails primaryIndexDetails =
+ (Index.ValueIndexDetails) primaryIndexInfo.index.getIndexDetails();
+ int[] projectionList = new int[skNames.size() + primaryIndexDetails.getKeyFieldNames().size()];
for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
outColumns[i] = primaryIndexInfo.rDesc.getFieldCount() + i;
}
@@ -378,7 +385,7 @@
for (int i = 0; i < secondaryFieldAccessEvalFactories.length; i++) {
projectionList[projCount++] = primaryIndexInfo.rDesc.getFieldCount() + i;
}
- for (int i = 0; i < primaryIndexInfo.index.getKeyFieldNames().size(); i++) {
+ for (int i = 0; i < primaryIndexDetails.getKeyFieldNames().size(); i++) {
projectionList[projCount++] = i;
}
IPushRuntime assignOp =
@@ -632,6 +639,7 @@
public SecondaryIndexInfo(PrimaryIndexInfo primaryIndexInfo, Index secondaryIndex) {
this.primaryIndexInfo = primaryIndexInfo;
this.secondaryIndex = secondaryIndex;
+ Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
CcApplicationContext appCtx =
(CcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext();
@@ -639,11 +647,11 @@
primaryIndexInfo.dataset, secondaryIndex.getIndexName(), nodes);
fileSplitProvider = new ConstantFileSplitProvider(splits);
secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
- primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
- secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
+ primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes, secondaryIndexDetails
+ .getKeyFieldTypes().toArray(new IAType[secondaryIndexDetails.getKeyFieldTypes().size()]));
secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType, primaryIndexInfo.metaType,
- primaryIndexInfo.primaryKeyTypes,
- secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
+ primaryIndexInfo.primaryKeyTypes, secondaryIndexDetails.getKeyFieldTypes()
+ .toArray(new IAType[secondaryIndexDetails.getKeyFieldTypes().size()]));
rDesc = new RecordDescriptor(secondaryIndexSerdes, secondaryIndexTypeTraits);
insertFieldsPermutations = new int[secondaryIndexTypeTraits.length];
for (int i = 0; i < insertFieldsPermutations.length; i++) {
@@ -651,7 +659,7 @@
}
primaryKeyIndexes = new int[primaryIndexInfo.primaryKeyIndexes.length];
for (int i = 0; i < primaryKeyIndexes.length; i++) {
- primaryKeyIndexes[i] = i + secondaryIndex.getKeyFieldNames().size();
+ primaryKeyIndexes[i] = i + secondaryIndexDetails.getKeyFieldNames().size();
}
}
@@ -711,7 +719,8 @@
keyFieldNames.add(Arrays.asList(fieldNames[primaryKeyIndexes[i]]));
}
index = new Index(dataset.getDataverseName(), dataset.getDatasetName(), dataset.getDatasetName(),
- IndexType.BTREE, keyFieldNames, primaryKeyIndicators, keyFieldTypes, false, false, true,
+ IndexType.BTREE,
+ new Index.ValueIndexDetails(keyFieldNames, primaryKeyIndicators, keyFieldTypes, false), false, true,
MetadataUtil.PENDING_NO_OP);
List<String> nodes = Collections.singletonList(ExecutionTestUtil.integrationUtil.ncs[0].getId());
CcApplicationContext appCtx =
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
index bc2c110..2d64f79 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/TestUserActor.java
@@ -200,7 +200,7 @@
DataverseName dataverseName = dataset.getDataverseName();
String datasetName = dataset.getDatasetName();
try {
- lockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName);
+ lockUtil.createIndexBegin(lockManager, mdProvider.getLocks(), dataverseName, datasetName, null);
if (actionListener.isActive()) {
throw new RuntimeDataException(ErrorCode.CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY,
DatasetUtil.getFullyQualifiedDisplayName(dataverseName, datasetName) + ".index",
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
index f28e9bb..f87757c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestDataset.java
@@ -55,14 +55,14 @@
}
@Override
- public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider,
- int[] primaryKeyFieldPermutation, boolean isSink) throws AlgebricksException {
+ public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider, int[] keyFieldPermutation,
+ boolean isSink) throws AlgebricksException {
return new IPushRuntimeFactory() {
@Override
public IPushRuntime[] createPushRuntime(IHyracksTaskContext ctx) throws HyracksDataException {
- return new IPushRuntime[] { new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()),
- getDatasetId(), primaryKeyFieldPermutation, true,
- ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
+ return new IPushRuntime[] {
+ new CommitRuntime(ctx, new TxnId(ctx.getJobletContext().getJobId().getId()), getDatasetId(),
+ keyFieldPermutation, true, ctx.getTaskAttemptId().getTaskId().getPartition(), true) };
}
};
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
index 532337d..9818538 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBTreeResourceFactoryProvider.java
@@ -98,20 +98,21 @@
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
}
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
}
@@ -132,22 +133,23 @@
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
}
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IBinaryComparatorFactoryProvider cmpFactoryProvider =
metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
IBinaryComparatorFactory[] secondaryCmpFactories =
new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
}
@@ -166,14 +168,14 @@
if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.BLOOM_FILTER_FIELDS;
} else {
- return new int[] { index.getKeyFieldNames().size() };
+ return new int[] { ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size() };
}
} else if (index.getIndexType() == IndexType.BTREE || index.getIndexType() == IndexType.RTREE) {
// secondary btrees and rtrees do not have bloom filters
return null;
} else {
// inverted indexes have bloom filters on deleted-key btrees
- int numKeys = index.getKeyFieldNames().size();
+ int numKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
int[] bloomFilterKeyFields = new int[numKeys];
for (int i = 0; i < numKeys; i++) {
bloomFilterKeyFields[i] = i;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
index 55a1a80..8cd02ed 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/NullMissingTest.java
@@ -86,9 +86,11 @@
// Include only functions annotated with MissingNullInOutFunction
if (functionDescriptor.getClass().isAnnotationPresent(MissingNullInOutFunction.class)) {
- // We test all functions except record and cast functions, which requires type settings (we test them
- // in runtime tests).
- if (!className.contains("record") && !className.contains("Cast")) {
+ // We test all functions except record, cast and full-text contains functions,
+ // which requires type settings or argument settings.
+ // Instead, we test them in runtime tests.
+ if (!className.contains("record") && !className.contains("Cast")
+ && !className.contains("FullTextContains")) {
tests.add(new Object[] { getTestName(functionDescriptor.getClass()), functionDescriptor });
} else {
LOGGER.log(Level.INFO, "Excluding " + className);
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_previous_success.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_previous_success.sqlpp
index 041fc6d..f9e49db 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_previous_success.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_previous_success.sqlpp
@@ -23,7 +23,6 @@
*/
use custord;
-write output to asterix_nc1:"mdtest/transaction_verify_failure_previous_success.adm";
SELECT VALUE t
FROM Metadata.`Datatype` t
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_subsequent_no_execution.sqlpp b/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_subsequent_no_execution.sqlpp
index 16c9b26..023cdbc 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_subsequent_no_execution.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/metadata/queries/transaction/verify_failure_subsequent_no_execution.sqlpp
@@ -24,7 +24,6 @@
use custord;
-write output to asterix_nc1:"mdtest/transaction_verify_failure_subsequent_no_execution.adm";
SELECT VALUE idx
FROM Metadata.`Index` idx
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 84ce315..80bc98a 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -7,6 +7,8 @@
{ "DataverseName": "Metadata", "DatasetName": "Feed", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 10, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "FeedConnection", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 11, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "DatatypeDataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 12, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextConfig", "DatatypeDataverseName": "Metadata", "DatatypeName": "FullTextConfigRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FullTextConfigName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FullTextConfigName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Thu Oct 29 16:14:12 PDT 2020", "DatasetId": 16, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextFilter", "DatatypeDataverseName": "Metadata", "DatatypeName": "FullTextFilterRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "FullTextFilterName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "FullTextFilterName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Thu Oct 29 16:14:12 PDT 2020", "DatasetId": 17, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Function", "DatatypeDataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 7, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "DatatypeDataverseName": "Metadata", "DatatypeName": "IndexRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 4, "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Library", "DatatypeDataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "DatasetType": "INTERNAL", "GroupName": "MetadataGroup", "CompactionPolicy": "concurrent", "CompactionPolicyProperties": [ { "Name": "max-component-count", "Value": "30" }, { "Name": "min-merge-component-count", "Value": "3" }, { "Name": "max-merge-component-count", "Value": "10" }, { "Name": "size-ratio", "Value": "1.2" } ], "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ [ "DataverseName" ], [ "Name" ] ], "PrimaryKey": [ [ "DataverseName" ], [ "Name" ] ], "Autogenerated": false }, "Hints": {{ }}, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "DatasetId": 9, "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 9aa9269..d352a6f 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -29,6 +29,9 @@
{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextConfigRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextConfigName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Tokenizer", "FieldType": "string", "IsNullable": true, "IsMissable": false }, { "FieldName": "FullTextFilterPipeline", "FieldType": "FullTextFilterPipeline", "IsNullable": true, "IsMissable": false } ] } }, "Timestamp": "Mon Nov 23 02:42:02 PST 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextFilterPipeline", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Nov 23 02:42:02 PST 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FullTextFilterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextFilterName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FullTextFilterType", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Mon Nov 23 02:42:02 PST 2020" }
{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false, "IsMissable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Dependencies", "FieldType": "FunctionRecordType_Dependencies", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item_Item" }, "Timestamp": "Thu May 14 18:42:45 PDT 2020" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index bdb22c7..5a40088 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -7,6 +7,8 @@
{ "DataverseName": "Metadata", "DatasetName": "Feed", "IndexName": "Feed", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "FeedConnection", "IndexName": "FeedConnection", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FeedName" ], [ "DatasetName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "FeedPolicy", "IndexName": "FeedPolicy", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "PolicyName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextConfig", "IndexName": "FullTextConfig", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FullTextConfigName" ] ], "IsPrimary": true, "Timestamp": "Thu Oct 29 16:20:12 PDT 2020", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "FullTextFilter", "IndexName": "FullTextFilter", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "FullTextFilterName" ] ], "IsPrimary": true, "Timestamp": "Thu Oct 29 16:20:12 PDT 2020", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ], [ "Arity" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "DatasetName" ], [ "IndexName" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
{ "DataverseName": "Metadata", "DatasetName": "Library", "IndexName": "Library", "IndexStructure": "BTREE", "SearchKey": [ [ "DataverseName" ], [ "Name" ] ], "IsPrimary": true, "Timestamp": "Fri Oct 21 10:29:21 PDT 2016", "PendingOp": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
index 7031c72..27f8a71 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_selfjoin/metadata_selfjoin.1.adm
@@ -12,4 +12,6 @@
{ "dv1": "Metadata", "dv2": "Metadata" }
{ "dv1": "Metadata", "dv2": "Metadata" }
{ "dv1": "Metadata", "dv2": "Metadata" }
+{ "dv1": "Metadata", "dv2": "Metadata" }
+{ "dv1": "Metadata", "dv2": "Metadata" }
{ "dv1": "Metadata", "dv2": "Metadata" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index 8af4c67..a29dc48 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -524,37 +524,37 @@
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the BTree index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the BTree index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[age]" which is of type integer cannot be indexed using the RTree index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[age]" which is of type integer cannot be indexed using the RTree index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_3">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_4">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_5">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_6">
<output-dir compare="Text">none</output-dir>
- <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 1)</expected-error>
+ <expected-error>ASX1079: Compilation error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index. (in line 37, at column 33)</expected-error>
</compilation-unit>
</test-case>
</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..3960ee6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query1.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..a231ac7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query2.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
new file mode 100644
index 0000000..8fe3c15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-1/query3.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..c10c2a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..8e5cf1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query2.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
new file mode 100644
index 0000000..b8e9b49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-2/query3.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..91f6d8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query1.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..d4742ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query2.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND D.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..e3afa1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query3.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
new file mode 100644
index 0000000..f5d3c52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-3/query4.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D.date
+ FROM YelpCheckin C, C.checkin_times D
+) AS CD ON CD /*+ indexnl */ = M.date;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..1afd508
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query1.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..b911a20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query2.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND CT.checkin_times_id = 1
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
new file mode 100644
index 0000000..5a23cb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query3.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
new file mode 100644
index 0000000..c142d1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/join-unnest-queries/use-case-4/query4.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..f267701
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..6cbfdff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query2.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
new file mode 100644
index 0000000..d2dc752
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-1/query3.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE LEN(C.dates) > 0 AND
+ (EVERY D IN C.dates
+ SATISFIES D > "2016" AND D < "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..145f8f21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26" IN C.checkin_times.dates;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..65e15fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query2.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
new file mode 100644
index 0000000..24ae2e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-2/query3.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times.dates) > 0 AND
+ (EVERY D IN C.checkin_times.dates
+ SATISFIES D > "2016" AND D < "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..3c07b9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query1.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..c8d5c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query2.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES D.date = "2016-04-26" AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..52f533d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-3/query3.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times) > 0 AND
+ (EVERY D IN C.checkin_times
+ SATISFIES D.date BETWEEN "2016" AND "2017");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..1bd9564
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query1.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" IN D.dates;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..204e589
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME CT IN C.checkin_times
+SATISFIES (
+ SOME D IN CT.dates
+ SATISFIES D = "2016-04-26"
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
new file mode 100644
index 0000000..8b3fd61
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/use-case-4/query3.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+/* TODO (GLENN) Support checking for nested length clauses (i.e. EVERY, then EVERY). */
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE LEN(C.checkin_times) > 0 AND
+ (EVERY CT IN C.checkin_times
+ SATISFIES (
+ SOME D IN CT.dates
+ SATISFIES "2019-06-07" = D
+ ));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..d4bb5ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..ae997b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-1/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..602e924
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..0d94cce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-2/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..52d826a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..0755f55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..49949a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-3/query3.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..2e3c0a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query1.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..89dc790
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/use-case-4/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
new file mode 100644
index 0000000..660f62b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
new file mode 100644
index 0000000..ce7fad8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
new file mode 100644
index 0000000..80a6bcc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
new file mode 100644
index 0000000..3ea8ac2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-pk/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
new file mode 100644
index 0000000..b9970c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query1.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ "19:49:16" = D.time AND
+ "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
new file mode 100644
index 0000000..022d5ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-composite-sk/query2.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
new file mode 100644
index 0000000..9b36d62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query1.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D AND
+ C.business_id = "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
new file mode 100644
index 0000000..2cffd50
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/closed/with-filter-fields/query2.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
new file mode 100644
index 0000000..9e24961
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
new file mode 100644
index 0000000..bdc0089
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-1/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
new file mode 100644
index 0000000..2a8e0ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
new file mode 100644
index 0000000..6eaeb8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-2/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
new file mode 100644
index 0000000..ff8aaf2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
new file mode 100644
index 0000000..cf4dbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
new file mode 100644
index 0000000..e2bc739
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-3/query3.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ D.time = "19:49:16";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
new file mode 100644
index 0000000..4037a3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
new file mode 100644
index 0000000..0435f33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/use-case-4/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
new file mode 100644
index 0000000..2855490
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query1.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string) ;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
new file mode 100644
index 0000000..fb0bc10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-3-level-record-path/query2.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string) ;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
new file mode 100644
index 0000000..37ef7a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/with-composite-sk/query1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string);
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ "19:49:16" = D.time AND
+ "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
index 4d8c4ff..16cb966 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
index 96e43a9..5531908 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
index a5c5fe4..6a110d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -62,7 +62,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
index e465dc4..d51ba21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
@@ -63,7 +63,6 @@
create dataset Orders(OrderType) primary key oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
select element {'customer':c,'order':o}
from Orders as o,
Customers as c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
index 4580dab..4180df7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -67,7 +67,6 @@
create index fbmIdxAutId if not exists on FacebookMessages (`author-id-copy`) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
select element {'fbu-ID':user.id,'fbm-auth-ID':message.`author-id`,'uname':user.name,'message':message.message}
from FacebookUsers as user,
FacebookMessages as message
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
index 1c315f8..ec3a0c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
@@ -53,7 +53,6 @@
create index title_index on CSX (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
index 7ab67fa..69bacde 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.sqlpp
@@ -42,7 +42,6 @@
create index title_index on DBLP (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
index 845da2a..bc23fdb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
index db55b28..f4842fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
index ecac451..096a519 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
index 6354f25..293991c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
index d8a95cd..68ec1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
index 75ccb90..5cef060 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
index 2cf7dbb..e1e88f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
index 98f037e..91a11cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
index a88838f..87659ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
index 524b0a7..9fbb733 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
index f6a5120..f3ff0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
index cd4fae6..596e00f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
index 9efcfc6..7fd5825 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
index 3b3b5e1..eea7904 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
index d6d977c..f241c16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
index 985cde7..244f33e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
index 40ae7d1..483ae00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
index 8b9a588..d69564c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
index 14d1ce9..a58d505 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
index 4070114..ef6acd9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
index e3f42ca..8a277df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
index 8656ab8..0dd1929 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
index 38fad40..bf59ed0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
index e8d5627..d8d40f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
index ce68496..79fca89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
index 0ef6c12..903a162 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
index ee49511..b1c76ec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
@@ -41,7 +41,6 @@
create dataset Visitors(VisitorType) primary key vid;
-write output to asterix_nc1:"/tmp/fuzzy1.adm";
select element {'user_name':user.name,'visitor_name':visitor.name}
from Users as user,
Visitors as visitor
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
index 3f64adf..e939be8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.sqlpp
@@ -41,7 +41,6 @@
;
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/.adm";
select element name
from Customers as c
group by c.c_name as name
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
index 5205324..7f2ddc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
@@ -61,7 +61,6 @@
create dataset Orders(OrderType) primary key oid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/custorder.adm";
select element {'cid':cid,'cust':(from g select c),'cnt-orders':count(o),'orders': (from g select o)}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
index 67587dd..bcbfc44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
@@ -35,7 +35,6 @@
;
create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
SET `compiler.sort.parallel` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
index e13305d..e0d6fc5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
select element o
from DBLP as o
where test.contains(o.title,'Mu')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
index 0384667..1fae011 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
index e5b4885..aec7ae3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
index 578ef74..cb8f588 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
index da55c99..bb4ab10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
select element o
from DBLP as o
where test.`edit-distance-check`(o.authors,'Amihay Motro',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
index 91a809b..2d65719 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Amihay Motro') <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
index d684f0c..38ea6cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
index 5adcda0..79a8722 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
index 5b564db..e436255 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
index 3e024a0..668214a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
index 41ceade..5725318 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
index eb17ed4..a7aff1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
index 0ca8b00..c51de55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
index 412ffdb..bf23e2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
index 0f2fcf4..a00fbc8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
index 44f6164..d27123b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
index a4114d0..97f22b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
index cf8b4a7..41f1471 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
index 1bfd5f9..d23d947 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
index 0b8ec46..237ba0a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
set `compiler.sort.parallel` "true";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
index a1372d9..01319fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
index 89a3eb8..bb674c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
index d13c98b..0edcfb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.interests,['databases','computers','wine']) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
index d32a8f4..9155c05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
index 25568e0..081d8fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.interests,{{'computers','wine','databases'}},0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
index 2a26ee9..e4f17a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.interests,{{'computers','databases','wine'}}) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
index 2a1db6d..809e9f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
index 27157ff..d6ffaad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
index 6157fde..0ed509d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
index b0dc05a..ac0b485 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
index eb96ddd..5fadd63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
index c60c8fd..a28b732 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
index 8a5d1d6..98f723d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
index 1f725d6..3ca9826 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
index d2e3888..2d463eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
select element {'id':paper.id,'title':paper.title}
from DBLP as paper
where test.`edit-distance-check`(test.substring(paper.title,0,8),'datbase',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
index 248c243..e3f9eca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
index 8be4caa..cf28b69 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`gram-tokens`(paper.title,3,false),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
index 68106a5..44fe3aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
index 3b861b1..bdc57ed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
index 7083657..059c6a0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
index f5d9888..a12344f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
index a04a8ef..9acdc96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
index ceac8c6..20d5e42 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
index e4f3b8d..fb6906f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
index 56ede6d..e9bf36d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
index 7c71c3e..6edd59e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
set `compiler.internal.sanitycheck` "false";
select element {'aauthors':a.authors,'bauthors':b.authors,'ed':ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
index 30ea450..20b9944 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
index 4fdc637..9b77a0e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
index 0638b0c..d7dab19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
set `compiler.internal.sanitycheck` "false";
select element {'ainterests':a.interests,'binterests':b.interests,'ed':ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
index a003dee..09017d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
index 1372490..f904502 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
index 9530163..c0bb728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
@@ -43,7 +43,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
index 5a312b3..1cbb519 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
@@ -53,7 +53,6 @@
create index topicIIx on TweetMessages (referred_topics) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
select element {'tweet':t.tweetid,'similar-tweets':(
select element t2.tweetid
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
index 157d4e4..bdf0f1b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -65,7 +65,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
set `compiler.internal.sanitycheck` "false";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
index 691bcca..65e22d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -64,7 +64,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
set `compiler.sort.parallel` "true";
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
index 9db0bd7..b188a80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -65,7 +65,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
index 2348dd9..8213f63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -65,7 +65,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
index 5d49319..4280470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
index 6549c0f..86e8b35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
index 274a4c0..21a7790 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
index 2e7e6da..fb4dda6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b,'ed':ed[1]}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
index 246d50e..22952b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
index a12ab9a..e4da99c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
index fe2cde2..a07fa3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b,'ed':ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
index 522a4f8..41860ff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
index c3a34b3..1c59a63f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
index aac9ca7..5f8e680 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
index d3bcd49..e1bea4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
index 460f1cc..2b8f67a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
index 5e180f2..cd068f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
index 2b9aa31..75f6752 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
index a2dc8f2..5ae5989 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
index fbc230c..05a1168 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
index 5265874..3621c9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
index 1a9cd82..40d8ce5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
index 78645c7..aea0c5b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
index cffc18b..f1a293e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b,'ed':ed[1]}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
index d66efdf..8c38ae2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
index f8cd85b..5c11cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
index 4a78cee..60dab06 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
set `compiler.internal.sanitycheck` "false";
select element {'arec':a,'brec':b,'ed':ed}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
index 3fe35d6..e4a9e80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
index 8db10eeb..95f38ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
index c1f3af7..428c2fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
index 573f5e2..53a3d7a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
index c9cbd1c..80e6ebc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
index 6364474..a3c0a21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
index bd84396..c39c1fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
index b6bea6a..c191e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
index 3359746..fc5674b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
index 941d506..9cc33c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
index 5c46ddc..f5bea00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
index b874be2..fbcbcbe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
index 2a771e3..84df9e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
index 8d71bb6..41ced1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
index 476db19..48c0b1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
index 287ef06..2b0060b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
index bc7b1cd..5dd7713 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
index 77c26cf..44a85c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
index e2158e1..90db3fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
index c6c38b7..ca20a6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
index 50d2442..f302a64 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -62,7 +62,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
select element {'t1':t1.tweetid,'t2':t2.tweetid,'sim':sim[1]}
from TweetMessages as t1,
TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
index 0c2890a..8ddaf3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
index fdf6235..4af5645 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
index 8eb16d7..976b694 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
index b943eec..9a0043c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
index 37a119b..e258370 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
index 584522a..b1ecd72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
index 9b5e045..9369e03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
index 9c188bc..0f1d095 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
index 3fe8de3..2c8935d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
index d6254ab..7a793bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
index bc0f28c..3a94a93 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
index 3deb837..bba5234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
index 99a3497..eee1f9f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
index 4f07d69..e47250a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
index a11fec7..2117c18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
index c9b6d13..4a95b68 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
index db3a364..946e8f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
index 05a8936..c6df170 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
index b53584e..ff391bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -58,7 +58,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
set `compiler.sort.parallel` "true";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
index e682634..a2b8230 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
index 5d57ef6..b39c7ff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -61,7 +61,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
index acafb14..76dc578 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -61,7 +61,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
set `compiler.sort.parallel` "true";
select element c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
index e61386b..f6ea964 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgNgramIx on TweetMessages (nested.`message-text`) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
set `compiler.sort.parallel` "true";
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
index 292925a..02d5194 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -60,7 +60,6 @@
create index topicKeywordIx on TweetMessages (nested.`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`referred-topics`},'similar-tweets':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
index 245e3ac..05ba474 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -66,7 +66,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
index 2481d9e..d3cb934 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -66,7 +66,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
index 5b88828..af9aba0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
@@ -59,7 +59,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
index bb50b07..568e24b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
index dbf7c40..d14ec4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
@@ -59,7 +59,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
index df41b65..799ff00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
index 1d22a19..009b4cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
index 7e98edf..e31ef0d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
index b6407d5..aeaa21a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
index 3a5e14f..dfdc227 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -59,7 +59,6 @@
create index msgNgramIx on TweetMessages (nested.`message-text`:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
set `compiler.sort.parallel` "true";
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
index fe6c65d..b08b9fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
index 955f38c..d8b558d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
index eb6fe2c..04e7a34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
index a924e19..8d26337 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
index 19143d3..6177484 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -65,7 +65,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
index 0471760..9630f4a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -65,7 +65,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
index 001da3f..26d1ee6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
@@ -78,7 +78,6 @@
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/nested_loj.adm";
select element {'cust':c,'orders':orders}
from Customers as c
with orders as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
index 2514206..32d9053 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
@@ -54,7 +54,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
index abe7d59..5ba60b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -55,7 +55,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
index 1f0c190..ab13a5c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
@@ -54,7 +54,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
index fd29e2e..e8976ef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -55,7 +55,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
index dbb430a..5209f6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
index 060465d..bffd850 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
index 9caf6bd..a41be85 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
index d81fcd9..22dc400 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -54,7 +54,6 @@
create index msgNgramIx on TweetMessages (`message-text`:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
set `compiler.sort.parallel` "true";
set `compiler.internal.sanitycheck` "false";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
index fe6c65d..b08b9fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
index 1cb8237..e610b00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
index eb6fe2c..04e7a34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
index a924e19..8d26337 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
set `compiler.sort.parallel` "true";
select element {'title1':o1.title,'title2':o2.title}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
index 3fd898e..ff60253 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
index c59dba4..5abe1c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -60,7 +60,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp
index c8ad09c..523fc9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05.sqlpp
@@ -26,7 +26,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-05.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
index 8c3f6dd..069fbb6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-05.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp
index 0fdf599..34b10f5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-06.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
index 7ba14c9..3953793 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-06.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp
index 2ceb8d8..049b51f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-07.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
index 403a75e..4bbe9f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-07.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp
index 40c1bc3..320b608 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08.sqlpp
@@ -26,7 +26,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
index a82fe5b..224bcde 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp
index 510f2df..7d055d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09.sqlpp
@@ -26,7 +26,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
index 759ce3a..7d03e99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
@@ -25,7 +25,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.sqlpp
index 86a04a2..9fc84df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-04.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
index 1b585b6..01a4ea3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
@@ -26,7 +26,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-04.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.sqlpp
index 83ae02a..df39ed8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-05.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
index b14b9d6..1891dd7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-05.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.sqlpp
index b8cdcd0..9736773 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-06.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
index 570c37e..7697e27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-06.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.sqlpp
index b84796b..9df6808 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-07.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
index ffb1cf8..17eac95 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-07.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.sqlpp
index 9659d6c..bf31348 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
index 6de6121..6493149 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-08.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.sqlpp
index 39414d0..58b0b97 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-09.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
index 3f1a7847..9c02c46 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-09.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.sqlpp
index 18b75ef..e0aaad1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-10.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp
index e34f931..eb0c171 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105.sqlpp
@@ -28,7 +28,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-104.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
index 6049e5f..1c07be9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
@@ -28,7 +28,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-104.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
index afb08ca..330addd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-10.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.sqlpp
index f4d2a2d..5b44090 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-11.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
index b921fa3..21f2768 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
@@ -27,7 +27,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-11.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
index 7bebb74..8b83cf8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
@@ -49,7 +49,6 @@
;
create dataset Customers(CustomerType) primary key cid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/gby-using-orderby-desc.adm";
set `compiler.sort.parallel` "true";
select element {'name':name,'age':age}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
index b010a1f..6318ba9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
index 1c5bfaa..cbfec56 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
set `compiler.sort.parallel` "true";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
index deb8b4c..10f2183 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
index f5f47b6..c9ab66a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
set `compiler.sort.parallel` "true";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
index 5243efd..44f965d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
index 5ecea40..883a975 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
set `compiler.sort.parallel` "true";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
index fa77884..e7daca8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
index cb60838..ea7bfdb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
set `compiler.sort.parallel` "true";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
index b03b85b..e1ef5be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
@@ -47,7 +47,6 @@
create dataset Visitors(VisitorType) primary key vid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
select element {'uid':user.uid,'vid':visitor.vid}
from Users as user,
Visitors as visitor
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
index 7807359..ae9f845 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.sqlpp
@@ -38,7 +38,6 @@
;
create dataset DBLP(DBLPType) primary key id WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/fuzzyjoin_080.adm";
select element {'id':paperDBLP.id,'matches':matches}
from DBLP as paperDBLP
with matches as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
index 3ee5774..d36c40d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
@@ -47,7 +47,6 @@
load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
set `compiler.sort.parallel` "true";
select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
index 2a61fa5..af73da4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
index a30f22c..cc0f899 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
index 3e1c881..3b9eda7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
set `compiler.sort.parallel` "true";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
index f87832e..6bde012 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -47,7 +47,6 @@
create index rtree_index on MyData2 (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
index afe7a72..e60b46a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -45,7 +45,6 @@
create index rtree_index on MyData (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
select element {'a':a,'b':b}
from MyData as a,
MyData as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
index c14a25e..a8a11b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
set `compiler.sort.parallel` "true";
select element o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
index 6235842..3b8ac91 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.sqlpp
@@ -28,7 +28,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
index 0a53f64..01dc63a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
@@ -28,7 +28,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/statement-params_statement-params-01.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
index 71e9f3c..5a87623 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
@@ -28,7 +28,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
index 9019f68..add9560 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
@@ -28,7 +28,6 @@
create dataverse test;
use test;
-write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
create type TestOpenType as open {
c_id: int64
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
index fc99d0b..8aab30e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
index 96e43a9..5531908 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_01.sqlpp
index f00eeeb..17cd88b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_01.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_02.sqlpp
index c12df2e..b84eae9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_02.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_03.sqlpp
index fa95763..e8bfa2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-join_03.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
index fdc5d9b..022bf07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
index 285affd2..776a1a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
index 61268a4..791db1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
index 83feb01..e569dcf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_04.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
index dab1456..512c9c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_05.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
index 54a57dd..e14c5fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
@@ -37,7 +37,6 @@
create dataset Names(Name) primary key fname,lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_06.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-multipred.sqlpp
index 7cf8561..623d3fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-multipred.sqlpp
@@ -65,7 +65,6 @@
create dataset Orders(OrderType) primary key oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-multipred.adm";
select element {'customer':c,'order':o}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
index 37fc73e..95c368b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
index 31126b6..d7384d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join-neg_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
index 3822746..77d83a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
index fb1bf93..91bb2b3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_03.sqlpp
index 9d1aaf3..496d0d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_03.sqlpp
@@ -63,7 +63,6 @@
create dataset Orders(OrderType) primary key oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_04.adm";
select element {'customer':c,'order':o}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_04.sqlpp
index e465dc4..d51ba21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_04.sqlpp
@@ -63,7 +63,6 @@
create dataset Orders(OrderType) primary key oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
select element {'customer':c,'order':o}
from Orders as o,
Customers as c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_05.sqlpp
index 75f6bdd..b6d1da5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-equi-join_05.sqlpp
@@ -51,7 +51,6 @@
create dataset Customers(CustomerType) primary key cid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_06.adm";
select element {'customer1':c1,'customer2':c2}
from Customers as c1,
Customers as c2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
index 64433e9..3b92465 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
index 30c5406..c6618cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-ge-join_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
index 67e8208..a63fa11 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
index 84a84c0..5a18107 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-gt-join_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
index a80c2ca..8444c06 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
index 66ac893..a7ebe01 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-le-join_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
index 3c719d8..4f511c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_01.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
index d32d8a1..ecc0391 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/primary-lt-join_02.sqlpp
@@ -38,7 +38,6 @@
create dataset test1.DsTwo(TestType) primary key key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multiindex.sqlpp
index 4580dab..4180df7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multiindex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -67,7 +67,6 @@
create index fbmIdxAutId if not exists on FacebookMessages (`author-id-copy`) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
select element {'fbu-ID':user.id,'fbm-auth-ID':message.`author-id`,'uname':user.name,'message':message.message}
from FacebookUsers as user,
FacebookMessages as message
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multipred.sqlpp
index 0fb1462..67c58c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join-multipred.sqlpp
@@ -55,7 +55,6 @@
create index title_index on DBLP (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multipred.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_01.sqlpp
index 30352db..93d306d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_01.sqlpp
@@ -53,7 +53,6 @@
create index title_index on DBLP (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_02.sqlpp
index 1c315f8..ec3a0c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_02.sqlpp
@@ -53,7 +53,6 @@
create index title_index on CSX (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_03.sqlpp
index 7ab67fa..69bacde 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index-join/secondary-equi-join_03.sqlpp
@@ -42,7 +42,6 @@
create index title_index on DBLP (title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-01.sqlpp
index 9e4f892..bf27d14 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-01.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-01.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-02.sqlpp
index f2e401e..9d7a61a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-02.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-02.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-03.sqlpp
index ad6db11..ac7cc99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-03.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-03.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-04.sqlpp
index b5ea3a0..ea3e82e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-04.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-04.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-05.sqlpp
index b237a63..00fbe0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-05.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-05.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-06.sqlpp
index e653b33..d05dbfb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-06.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-06.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-07.sqlpp
index bef5f61..f426bd5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-07.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-07.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-08.sqlpp
index d8f267e..40e30bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-08.sqlpp
@@ -28,7 +28,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-08.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-09.sqlpp
index b2c4c89..ed2a4c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-09.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-09.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-09.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-10.sqlpp
index 3d19624..f83eff1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-10.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-10.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-10.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-11.sqlpp
index 4838fea..e9786c8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-11.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-11.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-11.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-12.sqlpp
index 5c5f01b..e2814f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-12.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-12.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-12.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-13.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-13.sqlpp
index 90719de..3ae6a62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-13.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-13.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-13.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-14.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-14.sqlpp
index 14d9178..40696b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-14.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-14.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-14.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-15.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-15.sqlpp
index f182790..31cdf2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-15.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-15.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-15.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-16.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-16.sqlpp
index 0298ab5..5f849f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-16.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-16.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-16.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-17.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-17.sqlpp
index bb5de9e..afdefb5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-17.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-17.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-17.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-18.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-18.sqlpp
index 0f02205..c0785d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-18.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-18.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-18.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-19.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-19.sqlpp
index e095a0b..260c7ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-19.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-19.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-19.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-20.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-20.sqlpp
index 1735713..c1b2810 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-20.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-20.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-20.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-21.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-21.sqlpp
index 95a9407..600ffac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-21.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-21.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-21.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-22.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-22.sqlpp
index 3fd48d5..2274dfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-22.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-22.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-22.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-23.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-23.sqlpp
index 72a2a480..f87b149 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-23.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-23.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-23.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-24.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-24.sqlpp
index 0b8a9aa..1b4af59 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-24.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-24.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-24.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-25.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-25.sqlpp
index fb855e1..b63ef80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-25.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-25.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-25.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-26.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-26.sqlpp
index e187f4a..fc2367e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-26.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-26.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-26.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-27.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-27.sqlpp
index 8b83fff..778a67c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-27.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-27.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-27.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-28.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-28.sqlpp
index 388f4a9..de1b4c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-28.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-28.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-28.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-29.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-29.sqlpp
index 5b6d1fc..67e7054 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-29.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-29.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-29.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-30.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-30.sqlpp
index 9dfe2b6..d708889 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-30.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-30.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-30.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-31.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-31.sqlpp
index 2c36adb..bc5c463 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-31.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-31.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-31.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-32.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-32.sqlpp
index f61fee1..3822bfb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-32.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-primary-32.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-32.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-33.sqlpp
index 845da2a..bc23fdb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-33.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-33.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-34.sqlpp
index db55b28..f4842fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-34.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-34.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-35.sqlpp
index ecac451..096a519 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-35.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-35.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-36.sqlpp
index 6354f25..293991c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-36.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-36.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-37.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-37.sqlpp
index 5a044bc..cf73784 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-37.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-37.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-37.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-38.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-38.sqlpp
index 154c912..a34412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-38.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-38.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-38.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-39.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-39.sqlpp
index 9e274b0..65886cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-39.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-39.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-39.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-40.sqlpp
index d8a95cd..68ec1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-40.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-40.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-41.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-41.sqlpp
index 0c9ab8d..b5cba41 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-41.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-41.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-41.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-42.sqlpp
index 75ccb90..5cef060 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-42.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-42.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-43.sqlpp
index 2cf7dbb..e1e88f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-43.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-43.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-44.sqlpp
index 98f037e..91a11cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-44.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-44.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-45.sqlpp
index a88838f..87659ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-45.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-45.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-46.sqlpp
index 524b0a7..9fbb733 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-46.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-46.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-47.sqlpp
index f6a5120..f3ff0d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-47.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-47.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-48.sqlpp
index cd4fae6..596e00f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-48.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-48.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-49.sqlpp
index 9efcfc6..7fd5825 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-49.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-49.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-50.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-50.sqlpp
index 4ad3c01..00cbe7b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-50.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-50.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-50.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-51.sqlpp
index 3b3b5e1..eea7904 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-51.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-51.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-52.sqlpp
index d6d977c..f241c16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-52.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-52.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-53.sqlpp
index 985cde7..244f33e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-53.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-53.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-54.sqlpp
index 40ae7d1..483ae00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-54.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-54.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-55.sqlpp
index 8b9a588..d69564c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-55.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-55.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-56.sqlpp
index 14d1ce9..a58d505 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-56.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-56.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-57.sqlpp
index 4070114..ef6acd9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-57.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-57.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-58.sqlpp
index e3f42ca..8a277df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-58.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-58.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-59.sqlpp
index 8656ab8..0dd1929 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-59.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-59.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-60.sqlpp
index 38fad40..bf59ed0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-60.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-60.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-61.sqlpp
index e8d5627..d8d40f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-61.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-61.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-62.sqlpp
index ce68496..79fca89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-62.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-62.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-63.sqlpp
index 0ef6c12..903a162 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-63.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-63.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-64.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-64.sqlpp
index 723a551..6e7b2b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-64.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-64.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-64.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-65.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-65.sqlpp
index 2de8f5b..344ba5b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-65.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-65.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-65.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-66.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-66.sqlpp
index 800beef..2c5edce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-66.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-66.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-66.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-67.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-67.sqlpp
index 89e6697..7817b8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-67.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/btree-index/btree-secondary-67.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-67.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/collocated.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/collocated.sqlpp
index ee49511..b1c76ec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/collocated.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/collocated.sqlpp
@@ -41,7 +41,6 @@
create dataset Visitors(VisitorType) primary key vid;
-write output to asterix_nc1:"/tmp/fuzzy1.adm";
select element {'user_name':user.name,'visitor_name':visitor.name}
from Users as user,
Visitors as visitor
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-complex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-complex.sqlpp
index ff91548..323c688 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-complex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-complex.sqlpp
@@ -40,7 +40,6 @@
load dataset DBLP using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`)) pre-sorted;
-write output to asterix_nc1:"rttest/consolidate-complex-selects.aql";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-simple.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-simple.sqlpp
index fcd5e68..4c6cda9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-simple.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/consolidate-selects-simple.sqlpp
@@ -41,7 +41,6 @@
;
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/consolidate-selects-simple.aql";
select element c
from Customers as c
where ((c.c_name = 'testname') and (c.c_address = 'testaddr') and (c.c_nationkey = 1) and (c.c_phone = '123456789'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/const-folding.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/const-folding.sqlpp
index 583c0b8..30cf2ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/const-folding.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/const-folding.sqlpp
@@ -23,5 +23,4 @@
use test;
-write output to asterix_nc1:"rttest/divide.adm";
{'a':(1 + 2),'b':['foo','bar'][0]}.b;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/count-tweets.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/count-tweets.sqlpp
index 9ec110a..a1fcb3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/count-tweets.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/count-tweets.sqlpp
@@ -34,7 +34,6 @@
create external dataset TwitterData(Tweet) using localfs((`path`=`asterix_nc1://data/twitter/smalltweets.txt`),("format"="adm"));
-write output to asterix_nc1:"/tmp/count-tweets.adm";
select element {'word':tok,'count':count(token)}
from TwitterData as t,
tokens as token
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/cust_group_no_agg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/cust_group_no_agg.sqlpp
index 3f64adf..e939be8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/cust_group_no_agg.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/cust_group_no_agg.sqlpp
@@ -41,7 +41,6 @@
;
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/.adm";
select element name
from Customers as c
group by c.c_name as name
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/denorm-cust-order.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/denorm-cust-order.sqlpp
index 5205324..7f2ddc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/denorm-cust-order.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/denorm-cust-order.sqlpp
@@ -61,7 +61,6 @@
create dataset Orders(OrderType) primary key oid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/custorder.adm";
select element {'cid':cid,'cust':(from g select c),'cnt-orders':count(o),'orders': (from g select o)}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/distinct_aggregate.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/distinct_aggregate.sqlpp
index 812e6d0..91afa2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/distinct_aggregate.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/distinct_aggregate.sqlpp
@@ -49,7 +49,6 @@
;
create dataset LineItems_q1(LineItemType) primary key l_orderkey,l_linenumber WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'count_suppkey':count(g)}
from (
select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'l_suppkey':l_suppkey}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/filter-nested.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/filter-nested.sqlpp
index 53d6cc7..f0df73b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/filter-nested.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/filter-nested.sqlpp
@@ -23,7 +23,6 @@
use test;
-write output to asterix_nc1:"rttest/filter-nested.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-dblp-csx.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-dblp-csx.sqlpp
index 51c137c..2e72ad1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-dblp-csx.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-dblp-csx.sqlpp
@@ -49,7 +49,6 @@
create dataset CSX(CSXType) primary key id WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/fj-dblp-csx.adm";
select element {'idDBLP':idDBLP,'idCSX':idCSX}
from DBLP as paperDBLP,
tokensDBLP as prefixTokenDBLP,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase1.sqlpp
index 98a8dce..7709771 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase1.sqlpp
@@ -45,7 +45,6 @@
create dataset Visitors(VisitorType) primary key vid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/rares03.adm";
select element {'uid':user.uid,'tokens':tokens}
from Users as user
with tokens as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase2-with-hints.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase2-with-hints.sqlpp
index 26af6b6..8b74a121 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase2-with-hints.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/fj-phase2-with-hints.sqlpp
@@ -38,7 +38,6 @@
;
create dataset DBLP_fuzzyjoin_078(DBLPType) primary key id WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/fuzzyjoin_078.adm";
select element {'id':paperDBLP.id,'tokens':tokensDBLP}
from DBLP_fuzzyjoin_078 as paperDBLP
with unrankedTokensDBLP as fuzzyjoin_078.`counthashed-word-tokens`(paperDBLP.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/hashjoin-with-unnest.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/hashjoin-with-unnest.sqlpp
index 4f5263c1..86626da 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/hashjoin-with-unnest.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/hashjoin-with-unnest.sqlpp
@@ -38,7 +38,6 @@
create dataset t2(TestType) primary key id;
-write output to asterix_nc1:"rttest/hahsjoin-with-unnest.adm";
select element {'t1':m,'t2':n}
from t1 as m,
t2 as n,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inline-funs.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inline-funs.sqlpp
index 720108a..3b3b8ac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inline-funs.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inline-funs.sqlpp
@@ -23,7 +23,6 @@
use test;
-write output to asterix_nc1:"/tmp/inline_funs.adm";
declare function f1() {
(1 + test.f2())
};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
index c806296..6b0b8e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inlined_q18_large_volume_customer.sqlpp
@@ -78,7 +78,6 @@
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/inlined_q18_large_volume_customer.adm";
select element {'c_name':c_name,'c_custkey':c_custkey,'o_orderkey':o_orderkey,'o_orderdate':o_orderdate,'o_totalprice':o_totalprice,'sum_quantity':inlined_q18_large_volume_customer.strict_sum((
select element j.l_quantity
from l as j
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/introhashpartitionmerge.sqlpp
index a9cf298..cceea07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/introhashpartitionmerge.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/introhashpartitionmerge.sqlpp
@@ -35,7 +35,6 @@
;
create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
select element token2.rank
from TOKENSRANKEDADM as token1,
(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains-panic.sqlpp
index e13305d..e0d6fc5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
select element o
from DBLP as o
where test.contains(o.title,'Mu')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains.sqlpp
index e5b4885..aec7ae3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-contains.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
index 1f04301..f8b3095 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check.sqlpp
index da55c99..bb4ab10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
select element o
from DBLP as o
where test.`edit-distance-check`(o.authors,'Amihay Motro',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-panic.sqlpp
index 84eae49..db5ab45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance-panic.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-panic.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Amihay Motro') <= 5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance.sqlpp
index 91a809b..2d65719 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Amihay Motro') <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
index d684f0c..38ea6cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
index 5adcda0..79a8722 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard-check.sqlpp
index 5b564db..e436255 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard.sqlpp
index 3e024a0..668214a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ngram-jaccard.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
index 41ceade..5725318 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check.sqlpp
index 0ca8b00..c51de55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-panic.sqlpp
index 0f2fcf4..a00fbc8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance.sqlpp
index a4114d0..97f22b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
index 1bfd5f9..d23d947 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
index a1372d9..01319fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard-check.sqlpp
index 89a3eb8..bb674c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard.sqlpp
index d13c98b..0edcfb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/olist-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.interests,['databases','computers','wine']) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
index d32a8f4..9155c05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard-check.sqlpp
index 25568e0..081d8fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.interests,{{'computers','wine','databases'}},0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard.sqlpp
index 2a26ee9..e4f17a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/ulist-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.interests,{{'computers','databases','wine'}}) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-contains.sqlpp
index 2a1db6d..809e9f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-contains.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
index 6157fde..0ed509d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard-check.sqlpp
index b0dc05a..ac0b485 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard-check.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard.sqlpp
index eb96ddd..5fadd63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-basic/word-jaccard.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
index c60c8fd..a28b732 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
index 8a5d1d6..98f723d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
index 2cce890..e1b91f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
index 1f725d6..3ca9826 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
index d2e3888..2d463eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
select element {'id':paper.id,'title':paper.title}
from DBLP as paper
where test.`edit-distance-check`(test.substring(paper.title,0,8),'datbase',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
index ea53884..4fef571 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm";
select element {'id':paper.id,'title':paper.title}
from DBLP as paper,
test.`word-tokens`(paper.title) as word
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-let.sqlpp
index 248c243..e3f9eca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
index 8be4caa..cf28b69 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`gram-tokens`(paper.title,3,false),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
index 68106a5..44fe3aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let.sqlpp
index 7083657..059c6a0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-jaccard-check-let.sqlpp
index a04a8ef..9acdc96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ulist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ulist-jaccard-check-let.sqlpp
index ceac8c6..20d5e42 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ulist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-let.sqlpp
index e4f3b8d..fb6906f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
index 56ede6d..e9bf36d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
index c29de43..45bc472 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
select element {'aauthors':a.authors,'bauthors':b.authors,'ed':ed}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.sqlpp
index 767f6f5..4d4c029 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance.adm";
select element {'aauthors':a.authors,'bauthors':b.authors}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
index 1234824..3a734cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.sqlpp
index c050bc9..3477de8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.sqlpp
@@ -55,7 +55,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
index 4fdc637..9b77a0e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.sqlpp
index ca001a0..4c4f997 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.sqlpp
@@ -55,7 +55,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard.adm";
select element {'atitle':a.title,'btitle':b.title}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
index 40c8c5f..be28e73 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
select element {'ainterests':a.interests,'binterests':b.interests,'ed':ed}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.sqlpp
index 38c3cb9..7c44409c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance.adm";
select element {'ainterests':a.interests,'binterests':b.interests}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.sqlpp
index 4d8cfaa..779e026 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.sqlpp
index 6ce100b..7c9b678 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
index a003dee..09017d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.sqlpp
index e0a1a64..13ede78 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard.adm";
select element {'ainterests':a.interests,'binterests':b.interests}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.sqlpp
index 4a1bf06..e4abc4f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
index 1372490..f904502 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
select element {'ainterests':a.interests,'binterests':b.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.sqlpp
index 788c90f..0a20eb6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.sqlpp
@@ -56,7 +56,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard.adm";
select element {'ainterests':a.interests,'binterests':b.interests}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.sqlpp
index d5f0a42..384d857 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
index 9530163..c0bb728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
@@ -43,7 +43,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
select element {'atitle':a.title,'btitle':b.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard.sqlpp
index 89305c3..90dcaa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join-noeqjoin/word-jaccard.sqlpp
@@ -53,7 +53,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard.adm";
select element {'atitle':a.title,'btitle':b.title}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/issue741.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/issue741.sqlpp
index 5a312b3..1cbb519 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/issue741.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/issue741.sqlpp
@@ -53,7 +53,6 @@
create index topicIIx on TweetMessages (referred_topics) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
select element {'tweet':t.tweetid,'similar-tweets':(
select element t2.tweetid
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
index 8c2869f..59ca3e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -65,7 +65,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
select element {'id':t2.tweetid,'topics':t2.`message-text`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
index 9db0bd7..b188a80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -65,7 +65,6 @@
create index topicKeywordIx on TweetMessages (`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-contains.sqlpp
index ebec6c0..e377c6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-contains.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
DBLP as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_01.sqlpp
index 04cc151..9405c966 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_01.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_02.sqlpp
index a6ade32..dc365d1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_03.sqlpp
index ed8d6b6..2dab5c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_04.sqlpp
index 7a24772..5219dfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
select element {'arec':a,'brec':b,'ed':ed[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-contains.sqlpp
index ec44939..403ff8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance-contains.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-contains.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_01.sqlpp
index e2b15f4..fb6ee8c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_01.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_02.sqlpp
index 950a058..2938c15 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_03.sqlpp
index dc8ca3c..e78ee21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_04.sqlpp
index 0284ff4..2ed811b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
select element {'arec':a,'brec':b,'ed':ed}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
index 2021605..1d88e47 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on CSX (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
index 89f1354..5008ab0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
index d900a72..54b990f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
index 7c08462..d315e0c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
index aac9ca7..5f8e680 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
index d3bcd49..e1bea4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_01.sqlpp
index 60a1d7e..0006302 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_01.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_02.sqlpp
index 460f1cc..2b8f67a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_03.sqlpp
index 5e180f2..cd068f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_04.sqlpp
index 2b9aa31..75f6752 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_01.sqlpp
index 0057551..d910f38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_01.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_02.sqlpp
index a2dc8f2..5ae5989 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index on CSX (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_03.sqlpp
index fbc230c..05a1168 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_04.sqlpp
index 5265874..3621c9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -44,7 +44,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_01.sqlpp
index c14501c..8e7faa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_02.sqlpp
index f7e6047..e92fe44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_03.sqlpp
index 6435990..ff406ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_04.sqlpp
index 9f14368..8a0f634 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
select element {'arec':a,'brec':b,'ed':ed[1]}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_01.sqlpp
index 719314e..cbdb644 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_02.sqlpp
index 5ebb518..1638aeb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_03.sqlpp
index ac7e6ba..938175a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_04.sqlpp
index 0655746..b710479 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-edit-distance_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
select element {'arec':a,'brec':b,'ed':ed}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
index cecfbc9..1df1c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
index cd3d74f..4cd32e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
index a184602..e247926 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
index 39dde72..7931ecd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
index c1f3af7..428c2fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
index 573f5e2..53a3d7a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_01.sqlpp
index b67a1af..432ea71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_02.sqlpp
index c9cbd1c..80e6ebc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_03.sqlpp
index 6364474..a3c0a21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_04.sqlpp
index bd84396..c39c1fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_01.sqlpp
index deaecd7..a1f0de0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_02.sqlpp
index b6bea6a..c191e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_03.sqlpp
index 3359746..fc5674b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_04.sqlpp
index 941d506..9cc33c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/olist-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
index 2b42131..9ea5036 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
index 5c46ddc..f5bea00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
index b874be2..fbcbcbe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_01.sqlpp
index 66ac460..46df9f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_02.sqlpp
index 2a771e3..84df9e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_03.sqlpp
index 8d71bb6..41ced1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_04.sqlpp
index 476db19..48c0b1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard-check_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_01.sqlpp
index a2fd26d..ab234de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_01.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_02.sqlpp
index 287ef06..2b0060b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_02.sqlpp
@@ -55,7 +55,6 @@
create index interests_index on Customers2 (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_03.sqlpp
index bc7b1cd..5dd7713 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_03.sqlpp
@@ -53,7 +53,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_04.sqlpp
index 77c26cf..44a85c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/ulist-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index interests_index on Customers (interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
index 33e781d..49c9f5f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
index e2158e1..90db3fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
index c6c38b7..ca20a6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
index 50d2442..f302a64 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -62,7 +62,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
select element {'t1':t1.tweetid,'t2':t2.tweetid,'sim':sim[1]}
from TweetMessages as t1,
TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_01.sqlpp
index 79d4834..53f65ca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_01.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_02.sqlpp
index 0c2890a..8ddaf3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_03.sqlpp
index fdf6235..4af5645 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_04.sqlpp
index 8eb16d7..976b694 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_01.sqlpp
index 50e657a..b4f3482 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_01.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_02.sqlpp
index b943eec..9a0043c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_02.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index on CSX (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_03.sqlpp
index 37a119b..e258370 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_04.sqlpp
index 584522a..b1ecd72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/inverted-index-join/word-jaccard_04.sqlpp
@@ -42,7 +42,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_01.sqlpp
index 9b5e045..9369e03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_01.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_02.sqlpp
index 9c188bc..0f1d095 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/join-super-key_02.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/join-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/limit-issue353.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/limit-issue353.sqlpp
index 86ce33b..b51a642 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/limit-issue353.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/limit-issue353.sqlpp
@@ -51,7 +51,6 @@
create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
-write output to asterix_nc1:"/tmp/push_limit.adm";
select element {'l_returnflag':l.l_returnflag,'l_linestatus':l.l_linestatus,'l_shipmode':l.l_shipmode}
from LineItem as l
limit 2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_01.sqlpp
index 3fe8de3..2c8935d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_01.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_02.sqlpp
index d6254ab..7a793bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/loj-super-key_02.sqlpp
@@ -88,7 +88,6 @@
asterix_nc1,
asterix_nc2
;
-write output to asterix_nc1:"/tmp/loj-super-key_01.adm";
create dataset LineItems(LineItemType) primary key l_partkey,l_linenumber WITH {"node-group":{"name":"group1"}};
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
index b6d1a58..45806da 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
index c92304a..33dcb95 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.sqlpp
index 4e864f4..0990f04 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.sqlpp
index bf04345..9ffdfbe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.sqlpp
index c3c074a..23fdd55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
index 79a5709..9c1e97e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
index 824f076..1ff1e36 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
index eff456b..7d71424 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
index 726b78e..c2a3b8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_04.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
index 2841b8d..fe43b54 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_05.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
index dc9beef..18bf472 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.sqlpp
@@ -42,7 +42,6 @@
create dataset Names(Name) primary key nested.fname,nested.lname;
-write output to asterix_nc1:"rttest/btree-index-join_primary-composite-key-prefix-prefix-join_06.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.sqlpp
index 5d3c7ab..98b222b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.sqlpp
@@ -75,7 +75,6 @@
create dataset Orders(OrderType) primary key nested.oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-multipred.adm";
select element {'customer':c.nested,'order':o.nested}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
index 427eaef..d69cf9e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join-neg_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
index 5969055..3587023 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
index 8bc2165..5711557 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_02.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_02.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_03.sqlpp
index 3c73233..2e5db99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_03.sqlpp
@@ -73,7 +73,6 @@
create dataset Orders(OrderType) primary key nested.oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_04.adm";
select element {'customer':c.nested,'order':o.nested}
from Customers as c,
Orders as o
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_04.sqlpp
index 67e874c..3f764da 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_04.sqlpp
@@ -73,7 +73,6 @@
create dataset Orders(OrderType) primary key nested.oid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
select element {'customer':c.nested,'order':o.nested}
from Orders as o,
Customers as c
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_05.sqlpp
index 32be61e..623d272 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-equi-join_05.sqlpp
@@ -56,7 +56,6 @@
create dataset Customers(CustomerType) primary key nested.cid;
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_06.adm";
select element {'customer1':c1.nested,'customer2':c2.nested}
from Customers as c1,
Customers as c2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
index 4e13cc4..a49b704 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-ge-join_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-ge-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
index c006a0d..f1c8479 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-gt-join_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-gt-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
index c36ed5b..b0be2bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-le-join_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-le-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
index cb4241d..60850d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/primary-lt-join_01.sqlpp
@@ -43,7 +43,6 @@
create dataset test1.DsTwo(TestType) primary key nested.key1;
-write output to asterix_nc1:"rttest/btree-index-join_primary-lt-join_01.adm";
select element x
from test1.DsOne as x,
test1.DsTwo as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
index 3bbf0e0..011455b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -77,7 +77,6 @@
create index fbmIdxAutId if not exists on FacebookMessages (nested.`author-id-copy`) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
select element {'fbu-ID':user.nested.id,'fbm-auth-ID':message.nested.`author-id`,'uname':user.nested.name,'message':message.nested.message}
from FacebookUsers as user,
FacebookMessages as message
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.sqlpp
index 2c0a77f..7654857 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.sqlpp
@@ -65,7 +65,6 @@
create index title_index on DBLP (nested.title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multipred.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.sqlpp
index 67c5afb..8ee3b91 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.sqlpp
@@ -63,7 +63,6 @@
create index title_index on DBLP (nested.title) type btree;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-01.sqlpp
index af9ce63..9b88e9a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-01.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-01.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-02.sqlpp
index c653731..02592dd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-02.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-02.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-03.sqlpp
index 689b3d7..93b02b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-03.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-03.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-04.sqlpp
index 56b6f37..f3655e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-04.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-04.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-05.sqlpp
index 1223043..99057a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-05.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-05.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-06.sqlpp
index d530d0e..c1e0b66 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-06.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-06.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-07.sqlpp
index deee36b..f18816c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-07.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-07.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-08.sqlpp
index 0c21eaa..fa34725 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-08.sqlpp
@@ -28,7 +28,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-08.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-09.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-09.sqlpp
index f343db9..8d050eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-09.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-09.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-09.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-10.sqlpp
index dedde55..736ad3f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-10.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-10.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-10.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-11.sqlpp
index 2efc179..c538f27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-11.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-11.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-11.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-12.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-12.sqlpp
index 8f61a7f..41c1862 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-12.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-12.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-12.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-13.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-13.sqlpp
index 85c6606..128d38a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-13.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-13.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-13.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-14.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-14.sqlpp
index 8662c7b..95245fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-14.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-14.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-14.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-15.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-15.sqlpp
index 4222b7f..f1f789e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-15.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-15.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-15.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-16.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-16.sqlpp
index 63a6221..4795793 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-16.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-16.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-16.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-17.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-17.sqlpp
index e41e972..56b9e31 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-17.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-17.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-17.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-18.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-18.sqlpp
index 33bd030..892ae38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-18.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-18.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-18.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-19.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-19.sqlpp
index 886da30..2ba2414 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-19.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-19.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-19.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-20.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-20.sqlpp
index 88f8290..813a892 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-20.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-20.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-20.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-21.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-21.sqlpp
index 818f373..5103c71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-21.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-21.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-21.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-22.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-22.sqlpp
index cbbf38c..73fe94e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-22.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-22.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-22.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-23.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-23.sqlpp
index 3312363..0585602 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-23.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-23.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-23.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-24.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-24.sqlpp
index 4617096..204d9ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-24.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-24.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-24.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-25.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-25.sqlpp
index e1ce9bc..bfc0e55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-25.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-25.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-25.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-26.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-26.sqlpp
index 34f7225..84323d9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-26.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-26.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-26.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-27.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-27.sqlpp
index a5daa7f..111ea2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-27.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-27.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-27.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-28.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-28.sqlpp
index 426fb73..be135a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-28.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-28.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-28.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-29.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-29.sqlpp
index 7792ede..ce9f6b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-29.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-29.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-29.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-30.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-30.sqlpp
index 0328442..a35503a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-30.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-30.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-30.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-31.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-31.sqlpp
index 116e519..5486255 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-31.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-31.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-31.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-32.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-32.sqlpp
index 9bb3e6e..eeef81f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-32.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-primary-32.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-32.adm";
create type test.TestTypetmp as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-33.sqlpp
index 062adc8..052aca7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-33.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-33.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-31.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-34.sqlpp
index 28f3f58..7c13e7a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-34.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-34.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-32.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-35.sqlpp
index b66df57..41fd2dc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-35.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-35.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-33.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-36.sqlpp
index 7c1a0be..0f014d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-36.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-36.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-37.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-37.sqlpp
index 7d8eb2c..ee9dcb3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-37.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-37.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-38.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-38.sqlpp
index fe8b51c..aa8018f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-38.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-38.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-39.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-39.sqlpp
index 6967efb..afb05c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-39.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-39.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-37.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-40.sqlpp
index d27d758..e6d7e96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-40.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-40.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-38.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-41.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-41.sqlpp
index 60b47c4..a4ad3e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-41.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-41.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-39.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-42.sqlpp
index 6746e1e..7379a27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-42.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-42.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-43.sqlpp
index cd5fbd0..85a6120 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-43.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-43.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-41.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-44.sqlpp
index 795a72a..f884a8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-44.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-44.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-45.sqlpp
index 3df1d41..30b58a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-45.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-45.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-46.sqlpp
index 1c1e684..6f2f293 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-46.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-46.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-47.sqlpp
index 31d2013..fd4594e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-47.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-47.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-48.sqlpp
index 474a719..cc350dc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-48.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-48.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-49.sqlpp
index bfc1ea5..bcc3ca5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-49.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-49.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-50.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-50.sqlpp
index 6c4b63e..19977ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-50.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-50.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-51.sqlpp
index bc9fdaf..981eee7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-51.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-51.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-52.sqlpp
index 9f3682c..c56ec91 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-52.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-52.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-50.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-53.sqlpp
index 3b35ddf..1f38513 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-53.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-53.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-54.sqlpp
index 9ada29c..3744812 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-54.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-54.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-55.sqlpp
index a29b0a7..bda6187 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-55.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-55.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-56.sqlpp
index 7592325..650c9df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-56.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-56.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-57.sqlpp
index 61ad846..c4f7813 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-57.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-57.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-58.sqlpp
index f527712..6d99073 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-58.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-58.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-56.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-59.sqlpp
index 01bd660..a93dd6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-59.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-59.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-60.sqlpp
index 85e778e..6530e22 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-60.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-60.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-58.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-61.sqlpp
index dfe6a06..5540edf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-61.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-61.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-62.sqlpp
index baf2887..3df7a21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-62.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-62.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-63.sqlpp
index 405a8ae..3548f60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-63.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/btree-index/btree-secondary-63.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.sqlpp
index 0062d60..6a08d03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Mu')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains.sqlpp
index ba8cc73..4c9b543 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-contains.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
index 4b2abe6..32278d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
index 1bf50d0..e950b67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
select element o
from DBLP as o
where test.`edit-distance-check`(o.nested.authors,'Amihay Motro',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
index df59b6f..582d5bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-panic.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.nested.authors,'Amihay Motro') <= 5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.sqlpp
index 8649d19..c3446fd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.nested.authors,'Amihay Motro') <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
index 5898c33..e44a879 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
index b72c853..fc010d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.sqlpp
index 08b09e6..f2d8d8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.sqlpp
index fa1a246..495c367 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
index 81e7ae9..0890f5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.sqlpp
index 8c113d1..79c8f85 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
select element c
from Customers as c
where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.sqlpp
index 74d2796..9ef7f89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.sqlpp
index 2103dc6..0891786 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
select element c
from Customers as c
where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
index 53fe1ab..9d1e3b8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -58,7 +58,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
index fb4b646..a1231b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -58,7 +58,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.sqlpp
index 6d7e181..b021499 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -58,7 +58,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.nested.interests,['databases','computers','wine'],0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard.sqlpp
index e148b25..85c12ff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/olist-jaccard.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.nested.interests,['databases','computers','wine']) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
index 507ae3d..b9bab67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.sqlpp
index f4eb512..67ad2a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where test.`similarity-jaccard-check`(c.nested.interests,{{'computers','wine','databases'}},0.700000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.sqlpp
index fa54a39..4dcace1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
select element c
from Customers as c
where (test.`similarity-jaccard`(c.nested.interests,{{'computers','databases','wine'}}) >= 0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-contains.sqlpp
index 09b06b2..8a87042 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-contains.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
index 9d5fa37..8f6e938 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.sqlpp
index 8296b81..7ae7ddd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard.sqlpp
index 24db418..65d55a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-basic/word-jaccard.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
index 25b6fca..8e71116 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with eda as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',3),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
index 39c1276..cebcaa9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with edb as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
index b610925..5cc273e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
index 3a30c4f..0b087ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
index 2adcbb3..64b81af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
select element {'id':paper.nested.id,'title':paper.nested.title}
from DBLP as paper
where test.`edit-distance-check`(test.substring(paper.nested.title,0,8),'datbase',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
index 16b4f91..f135468 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm";
select element {'id':paper.nested.id,'title':paper.nested.title}
from DBLP as paper,
test.`word-tokens`(paper.nested.title) as word
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
index e1f79cb..7d3e74a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -49,7 +49,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
index db62898..75f472d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -49,7 +49,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`gram-tokens`(paper.nested.title,3,false),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
index fe78dbf..0cae971 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -61,7 +61,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.sqlpp
index 965d567..84ad4b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -61,7 +61,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
select element c
from Customers as c
with ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.sqlpp
index fe3f716..1a1754c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -59,7 +59,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.nested.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.sqlpp
index af0beed..4399307 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -61,7 +61,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
select element c
from Customers as c
with jacc as test.`similarity-jaccard-check`(c.nested.interests,['databases','computers','wine'],0.700000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.sqlpp
index 6666dca..1e81fa3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -47,7 +47,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
index 4bfed40..09e4bfa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -47,7 +47,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.nested.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
index bd41d46..0d43747 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -60,7 +60,6 @@
create index msgNgramIx on TweetMessages (nested.`message-text`) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
index be025d8..bc0b55c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -60,7 +60,6 @@
create index topicKeywordIx on TweetMessages (nested.`referred-topics`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`referred-topics`},'similar-tweets':(
select element {'id':t2.nested.tweetid,'topics':t2.nested.`referred-topics`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
index 5a540ce..0ebf194 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
index 0621d0c..414b57a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-contains.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
index 4cf643a..b623f02 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
select element {'aauthors':a.nested.authors,'bauthors':b.nested.authors,'ed':ed}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.sqlpp
index fd7d774..0f1ddfa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index on DBLP (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
index 4b9264c..dbf43e5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index on CSX (nested.authors) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
index b81b98c..8be00c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index on CSX (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
index 17c1b84..c829366 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index on CSX (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.sqlpp
index 2b5099a..a29b6a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on DBLP (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
select element {'atitle':a.nested.title,'btitle':b.nested.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.sqlpp
index a400a36..73dcb44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index on CSX (nested.title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.sqlpp
index bf280e1..0cabb05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.sqlpp
index 89626aa..f6dd862 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.sqlpp
@@ -62,7 +62,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
select element {'ainterests':a.nested.interests,'binterests':b.nested.interests,'ed':ed}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.sqlpp
index d38f25f..338da50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
index 7942f82..5569e82 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
index a56ef5c..977585f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.sqlpp
index be86585..330ab3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.sqlpp
index 4131908..973fe6a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.sqlpp
@@ -62,7 +62,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
select element {'ainterests':a.nested.interests,'binterests':b.nested.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.sqlpp
index c860a4a..c1bd5ff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
index 2dffabf..bc97f45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.7f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.sqlpp
index 2d4c31f..c0f3413 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.sqlpp
index 6676599..ff7e49c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.sqlpp
@@ -62,7 +62,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
select element {'ainterests':a.nested.interests,'binterests':b.nested.interests,'jacc':jacc}
from Customers as a,
Customers as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.sqlpp
index 45bcd7a..2baa993 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.sqlpp
@@ -60,7 +60,6 @@
create index interests_index on Customers (nested.interests) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_01.adm";
select element {'arec':a,'brec':b}
from Customers as a,
Customers2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
index 07402e2..f31bb72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
index ff005fd..7150ad4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -67,7 +67,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
select element {'t1':t1.nested.tweetid,'t2':t2.nested.tweetid,'sim':sim[1]}
from TweetMessages as t1,
TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.sqlpp
index 95e9bbb..1237b9c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.sqlpp
index 4fe09b9..cd49a51 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.sqlpp
@@ -48,7 +48,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
select element {'atitle':a.nested.title,'btitle':b.nested.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard_01.sqlpp
index 3b2d0e2..65766ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/inverted-index-join/word-jaccard_01.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index on DBLP (nested.title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
index 1b3582e..0791493 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
@@ -66,7 +66,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
index 6adb262..f584bc2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -66,7 +66,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.sqlpp
index 75f9de3..4089355 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.sqlpp
@@ -52,7 +52,6 @@
create index rtree_index on MyData1 (nested.point) type rtree;
-write output to asterix_nc1:"rttest/index-join_rtree-spatial-intersect-point.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.sqlpp
index 9d09f48..36b8a6b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -52,7 +52,6 @@
create index rtree_index on MyData2 (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.sqlpp
index fa3ff7b..ed703b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -50,7 +50,6 @@
create index rtree_index on MyData (nested.point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
select element {'a':a,'b':b}
from MyData as a,
MyData as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
index 4e597c4..eac8fc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
@@ -59,7 +59,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
index a153cd0..50acf4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
index c2ca7ce..bfc8888 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
@@ -59,7 +59,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
index d7a3be3..a558315 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
@@ -60,7 +60,6 @@
create index msgCountBIx on TweetMessages (nested.countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.sqlpp
index a7a3c3f..fc2a18b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.sqlpp
index c94fc26..c6d885c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.sqlpp
index 3881d81..f4c663d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
index 61a7672..6b2507e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
index 0d617a2..1853324 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
index 6c1a2a4..fc316a0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
index 8c09b0b..711c722 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_04.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
index 84069b4..2b72b8d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_05.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
index 4822293..89b5fca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
@@ -43,7 +43,6 @@
create index Name_idx on Names (nested.fname:string?,lnested.name:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_06.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
index cfb465a..7143bce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -76,7 +76,6 @@
create index fbmIdxAutId if not exists on FacebookMessages (nested.`author-id-copy`:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
select element {'fbu-ID':user.nested.id,'fbm-auth-ID':message.nested.`author-id`,'uname':user.nested.name,'message':message.nested.message}
from FacebookUsers as user,
FacebookMessages as message
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.sqlpp
index 92599bb..ab57e60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.sqlpp
@@ -64,7 +64,6 @@
create index title_index on DBLP (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multipred.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.sqlpp
index 6b207c5..6325285 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.sqlpp
@@ -62,7 +62,6 @@
create index title_index on DBLP (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.sqlpp
index fc67495..e9ee6bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.sqlpp
@@ -62,7 +62,6 @@
create index title_index on CSX (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.sqlpp
index c8b0dfd..478652b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.sqlpp
@@ -63,7 +63,6 @@
create index title_index_CSX on CSX (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.sqlpp
index c68c001..35ade41 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.sqlpp
@@ -46,7 +46,6 @@
create index title_index on DBLP (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_04.adm";
select element {'arec':a,'arec2':a2}
from DBLP as a,
DBLP as a2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.sqlpp
index 38ff226..7c02036 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.sqlpp
@@ -61,7 +61,6 @@
create index title_index on DBLP (nested.title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_05.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-33.sqlpp
index 5eedf99..f498287 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-33.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-33.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-39.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-34.sqlpp
index dafd317..d9f284c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-34.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-34.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-32.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-35.sqlpp
index 236d8a4..be4fa88 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-35.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-35.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-33.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-36.sqlpp
index 44a582b..44b1d4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-36.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-36.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-37.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-37.sqlpp
index 361af5f..31a5dc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-37.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-37.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-38.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-38.sqlpp
index a7cf700..0855c99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-38.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-38.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-39.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-39.sqlpp
index ed1bbcc..43ad20d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-39.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-39.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-37.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-40.sqlpp
index 8f4bab2..510f58a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-40.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-40.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-38.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-41.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-41.sqlpp
index 0d54ffc..cc12129 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-41.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-41.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-39.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-42.sqlpp
index f412a96..c430a70 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-42.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-42.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-43.sqlpp
index 5872426..a968f4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-43.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-43.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-41.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-44.sqlpp
index f7a4e51..d521fd3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-44.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-44.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-45.sqlpp
index e5eeb29..bbc2c42 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-45.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-45.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-46.sqlpp
index bf04ce0..ae355b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-46.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-46.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-47.sqlpp
index b2b23a1..b63f7a6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-47.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-47.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-48.sqlpp
index a3689e3..0f672c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-48.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-48.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-49.sqlpp
index f5d7566..b1cdd72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-49.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-49.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-50.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-50.sqlpp
index 171229e..ab7fc9c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-50.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-50.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-51.sqlpp
index 6c8d7ba..1d2466d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-51.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-51.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-52.sqlpp
index 68c46aa..6964cce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-52.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-52.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-50.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-53.sqlpp
index 91b932e..f20fbe7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-53.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-53.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-54.sqlpp
index 7b4ed5c..36e4eaf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-54.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-54.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-55.sqlpp
index a7a9ffa..fca97d9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-55.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-55.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-56.sqlpp
index bbb3b01..6757ab1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-56.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-56.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-57.sqlpp
index 4188230..370ddb4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-57.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-57.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-58.sqlpp
index d0d619f..55a27d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-58.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-58.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-56.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-59.sqlpp
index b454d71..00ebe46 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-59.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-59.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-60.sqlpp
index 62ed07e..ff60e34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-60.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-60.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-58.adm";
create type test.TestTypetmp as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-61.sqlpp
index c4bb152..c44881a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-61.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-61.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-62.sqlpp
index 198d361..5182551 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-62.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-62.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-63.sqlpp
index 85ff1d4..90c59b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-63.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/btree-index/btree-secondary-63.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
create type test.TestTypetmp as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.sqlpp
index 1d4e565..b1bc1db 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Mu')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.sqlpp
index ed08291..893bde7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
index 9377ea0..0b4c524 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
index e30ff08..ce64f37c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
select element o
from DBLP as o
where test.`edit-distance-check`(o.nested.authors,'Amihay Motro',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
index 55959f2..e867bb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-panic.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.nested.authors,'Amihay Motro') <= 5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.sqlpp
index 8c23d24..157e180 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.nested.authors,'Amihay Motro') <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
index 192ef74..1eb4740 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
index d6cfd59..feb3a06 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.sqlpp
index cbda7b4..d4325fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.sqlpp
index be07ae2..7fd412a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-contains.sqlpp
index 4a71613..c7db007 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-contains.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
select element o
from DBLP as o
where test.contains(o.nested.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
index 1170592..966409b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.sqlpp
index 5bfb33d..8b94bbc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.sqlpp
index e95a48d..0672d08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
index 8771118..36c5c22 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with eda as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',3),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
index 8bc92fe..f2de983 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with edb as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
index 7ce9153..8b9fd34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
index 05eca1e..6348faa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.nested.authors,'Amihay Motro',1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
index d202b43..1a92c02 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
select element {'id':paper.nested.id,'title':paper.nested.title}
from DBLP as paper
where test.`edit-distance-check`(test.substring(paper.nested.title,0,8),'datbase',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
index c37ebfd..f3ce375 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
@@ -46,7 +46,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm";
select element {'id':paper.nested.id,'title':paper.nested.title}
from DBLP as paper,
test.`word-tokens`(paper.nested.title) as word
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
index 5ff344c..fe11859 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.nested.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
index 964dfc0..b14fd32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -48,7 +48,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`gram-tokens`(paper.nested.title,3,false),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.sqlpp
index 19dd76d..dbdc744 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.nested.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
index 0a0abb2..2eb208a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -46,7 +46,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.nested.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
index 70e727e..ea14539 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -59,7 +59,6 @@
create index msgNgramIx on TweetMessages (nested.`message-text`:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.sqlpp
index cae1003..3baa6e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
CSX as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.sqlpp
index 9d87160..639b83e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
select element {'title1':o1.title,'title2':o2.title}
from CSX as o1,
DBLP as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.sqlpp
index a489041..63ed521 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
DBLP as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.sqlpp
index 2bc986c..2fe8d55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
CSX as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
index f4074f2..8078430 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.sqlpp
index 9f3f5c5..88adb82 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.sqlpp
index 0fbc905..9f286b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.sqlpp
index 87ece5a..0a7b2c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index_CSX on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.sqlpp
index c8add3e..31fad88 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.sqlpp
@@ -60,7 +60,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
index 6bf1c72..aae0e7b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-contains.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
index 0b7f72f..f20d4cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm";
select element {'aauthors':a.nested.authors,'bauthors':b.nested.authors,'ed':ed}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.sqlpp
index 661e010..1dcb68f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.sqlpp
index 3c4e9fe..c65c9b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.sqlpp
index e4d138e..559b196 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.sqlpp
index 08d9808..390dad2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index_CSX on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.sqlpp
index 19132a1..1b1a7da 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.sqlpp
@@ -60,7 +60,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
index a6f12a1..df40454 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
index 5326185..65bb72b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
@@ -61,7 +61,6 @@
create index ngram_index on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
index a2af936..3913e67 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -45,7 +45,6 @@
create index ngram_index on DBLP (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
index 68c4619..05d2d54 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
@@ -62,7 +62,6 @@
create index ngram_index_CSX on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
index d8d417b..31b2bdf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
@@ -60,7 +60,6 @@
create index ngram_index on CSX (nested.authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
index 51da2ab..6dcf890 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
index efe0ae7..b7b1b6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
index ccec588..9b18ee2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
index 379b7eb..25cd22d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index_CSX on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
index f1d7467..8f3ae2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.sqlpp
index 6a3ade7..7548a66 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.sqlpp
index cb5c545..07000ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.sqlpp
index 204f77e..aed22d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index_CSX on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.sqlpp
index 288be03..932e7db 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.sqlpp
@@ -49,7 +49,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
select element {'atitle':a.nested.title,'btitle':b.nested.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.sqlpp
index dc85897..4a6d5c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.sqlpp
index 2360a60..8aa597b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -63,7 +63,6 @@
create index ngram_index on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.sqlpp
index c65fe7f..c151500 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -47,7 +47,6 @@
create index ngram_index on DBLP (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.sqlpp
index bbbcada..6a879bf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -64,7 +64,6 @@
create index ngram_index_CSX on CSX (nested.title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
index dcd9bf9..ad76fde 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
index f469132..a49ed50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
index f3fbf21..9fc7a07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
index f63503d..4163035 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index_CSX on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
index 6896d96..16118f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -66,7 +66,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
select element {'t1':t1.nested.tweetid,'t2':t2.nested.tweetid,'sim':sim[1]}
from TweetMessages as t1,
TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.sqlpp
index 86aeb19..4f85a12 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.sqlpp
index 2f0c0ae..dcce98d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.sqlpp
index e4974c1..c27e0a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index_DBLP on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.sqlpp
index c7a5c1a..8488336 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.sqlpp
index bacc15e..e2653a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.sqlpp
@@ -47,7 +47,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
select element {'atitle':a.nested.title,'btitle':b.nested.title,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.sqlpp
index f9a31b3..1ab49fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.sqlpp
index e696010..a418aa5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.sqlpp
@@ -61,7 +61,6 @@
create index keyword_index on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from CSX as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.sqlpp
index 745949e..ea3502c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.sqlpp
@@ -45,7 +45,6 @@
create index keyword_index on DBLP (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.sqlpp
index b95c674..bd0d67b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.sqlpp
@@ -62,7 +62,6 @@
create index keyword_index on CSX (nested.title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
index 5a91cdc..ef0c806 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
@@ -65,7 +65,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
index 77d9ad0..b8f89b3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -65,7 +65,6 @@
create index msgTextIx on TweetMessages (nested.`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.sqlpp
index 75f9de3..4089355 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.sqlpp
@@ -52,7 +52,6 @@
create index rtree_index on MyData1 (nested.point) type rtree;
-write output to asterix_nc1:"rttest/index-join_rtree-spatial-intersect-point.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.sqlpp
index 9d09f48..36b8a6b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -52,7 +52,6 @@
create index rtree_index on MyData2 (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.sqlpp
index fa3ff7b..ed703b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -50,7 +50,6 @@
create index rtree_index on MyData (nested.point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
select element {'a':a,'b':b}
from MyData as a,
MyData as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.sqlpp
index 3993450..68cacdf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.sqlpp
@@ -53,7 +53,6 @@
create index rtree_index2 on MyData2 (nested.point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.sqlpp
index dc9059f..b73a9b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.sqlpp
@@ -51,7 +51,6 @@
create index rtree_index on MyData1 (nested.point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj2.sqlpp
index 001da3f..26d1ee6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj2.sqlpp
@@ -78,7 +78,6 @@
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/nested_loj.adm";
select element {'cust':c,'orders':orders}
from Customers as c
with orders as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj3.sqlpp
index 0b460b4..eec8ae8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj3.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/nested_loj3.sqlpp
@@ -89,7 +89,6 @@
create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/nested_loj.adm";
select element {'cust':c,'orders':orders}
from Customers as c
with orders as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
index bf9850a..9ee07e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.sqlpp
@@ -54,7 +54,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
index 0a94d56..aac98d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.sqlpp
@@ -55,7 +55,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
index bc517cf..e6f7ab1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.sqlpp
@@ -54,7 +54,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
index 0504b30..6897bc3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.sqlpp
@@ -55,7 +55,6 @@
create index msgCountBIx on TweetMessages (countB:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
select element {'tweetid2':t2.tweetid,'count2':t2.countB}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.sqlpp
index 1c2b803..772048b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.sqlpp
index 7008899..48ebd74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.sqlpp
index cfb31f7..3ebfbed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
index f0e8ff2..321081b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_01.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
index 400b7d1..ba675de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_02.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
index 39adc5c..ec8e50b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_03.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
index b403c1a..50a0930 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_04.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
index 22e6c4a..244e3f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_05.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
index 244f583..9e0d121 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.sqlpp
@@ -38,7 +38,6 @@
create index Name_idx on Names (fname:string?,lname:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_06.adm";
select element {'emp1':emp1,'emp2':emp2}
from Names as emp1,
Names as emp2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.sqlpp
index 88b0584..5465df5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -66,7 +66,6 @@
create index fbmIdxAutId if not exists on FacebookMessages (`author-id-copy`:integer?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
select element {'fbu-ID':user.id,'fbm-auth-ID':message.`author-id`,'uname':user.name,'message':message.message}
from FacebookUsers as user,
FacebookMessages as message
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.sqlpp
index 816815a..c390665 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.sqlpp
@@ -54,7 +54,6 @@
create index title_index on DBLP (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multipred.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.sqlpp
index 01bf0ab..3d41fec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.sqlpp
@@ -52,7 +52,6 @@
create index title_index on DBLP (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.sqlpp
index 750d100..26e7b43 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.sqlpp
@@ -52,7 +52,6 @@
create index title_index on CSX (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.sqlpp
index 3d33222..9c10549 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.sqlpp
@@ -41,7 +41,6 @@
create index title_index on DBLP (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.sqlpp
index d987e47..e1d86cd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.sqlpp
@@ -53,7 +53,6 @@
create index title_index on CSX (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.sqlpp
index c322c44..59d7901 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.sqlpp
@@ -52,7 +52,6 @@
create index title_index on DBLP (title:string?) type btree enforced;
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-33.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-33.sqlpp
index e4f0814..5f48b6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-33.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-33.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-31.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-34.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-34.sqlpp
index bcea4a7..ea91606 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-34.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-34.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-32.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-35.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-35.sqlpp
index 3d4f2d6..cf4176a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-35.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-35.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-33.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-36.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-36.sqlpp
index bebae85..a516608 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-36.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-36.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-37.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-37.sqlpp
index c67e089..bddc064 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-37.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-37.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-38.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-38.sqlpp
index 39fb3ae..ca0b20a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-38.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-38.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-39.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-39.sqlpp
index 357c6a3..9af21a4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-39.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-39.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-37.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-40.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-40.sqlpp
index bdead51..f420a36 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-40.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-40.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-38.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-41.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-41.sqlpp
index 6cb1afe..f2a96b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-41.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-41.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-39.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-42.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-42.sqlpp
index 1fd309d..5320b7c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-42.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-42.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-43.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-43.sqlpp
index 6c33486..df3b9c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-43.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-43.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-41.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-44.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-44.sqlpp
index f4e67d2..45d1921 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-44.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-44.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-45.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-45.sqlpp
index 225dceb..e7a8c42 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-45.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-45.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-46.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-46.sqlpp
index 61c58de..5b0a630 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-46.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-46.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-47.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-47.sqlpp
index 623e287..eaae4eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-47.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-47.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-48.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-48.sqlpp
index df9db9c..cc6cc2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-48.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-48.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-49.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-49.sqlpp
index 3726ebb..9a5c4fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-49.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-49.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-50.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-50.sqlpp
index 21a1e7c..d9f005a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-50.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-50.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-51.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-51.sqlpp
index ae5ca02..b791833 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-51.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-51.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-52.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-52.sqlpp
index 27c6807..325d782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-52.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-52.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-50.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-53.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-53.sqlpp
index 8e9aea5..1e53fb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-53.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-53.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-54.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-54.sqlpp
index 64be60b..3486bfe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-54.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-54.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-55.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-55.sqlpp
index 6d0f320..530c6d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-55.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-55.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-56.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-56.sqlpp
index 5ac912b..f1711ed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-56.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-56.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-57.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-57.sqlpp
index 2de2f34..6ef1cda 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-57.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-57.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-58.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-58.sqlpp
index de54406..80f68ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-58.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-58.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-56.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-59.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-59.sqlpp
index 4db7f17..061b685 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-59.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-59.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-60.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-60.sqlpp
index a0d20aa..338f70e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-60.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-60.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-58.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-61.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-61.sqlpp
index ea7e791..da311c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-61.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-61.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-62.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-62.sqlpp
index 7897e16..2f38237 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-62.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-62.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-63.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-63.sqlpp
index ae70c61..820e96d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-63.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/btree-index/btree-secondary-63.sqlpp
@@ -30,7 +30,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
create type test.TestType as
{
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.sqlpp
index 235d076..b9c06f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
select element o
from DBLP as o
where test.contains(o.title,'Mu')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.sqlpp
index d91f0a2..b03a0fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
index d4516ed..676c46f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.sqlpp
index 01183f7..d017060 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
select element o
from DBLP as o
where test.`edit-distance-check`(o.authors,'Amihay Motro',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.sqlpp
index dbb5814..5a24709 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-panic.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Amihay Motro') <= 5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.sqlpp
index 4c3ca04..6b27a40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Amihay Motro') <= 1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
index 4da110a..0b450c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
index 7c488d1..71091ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.sqlpp
index 240f372..19d58c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.sqlpp
index 28af41e..1f2b3a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false)) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-contains.sqlpp
index 2e26b98..9da3921 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-contains.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
select element o
from DBLP as o
where test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
index c905a53..93fb036 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.sqlpp
index fd0a151..f1f4acd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
select element o
from DBLP as o
where test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.sqlpp
index b3ef666..495bb5d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
where (test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
index e569be8..b17d6b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
index c2402bc..a5deea6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm";
select element o
from DBLP as o
with edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
index 30f29e2..a3f3e60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
index bbb060d..2398b8b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-let.adm";
select element o
from DBLP as o
with ed as test.`edit-distance-check`(o.authors,'Amihay Motro',1)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
index 0b6b09c..ea6699d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm";
select element {'id':paper.id,'title':paper.title}
from DBLP as paper
where test.`edit-distance-check`(test.substring(paper.title,0,8),'datbase',1)[0]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
index dec3c89..50f2bb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm";
select element {'id':paper.id,'title':paper.title}
from DBLP as paper,
test.`word-tokens`(paper.title) as word
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.sqlpp
index 8e726f1..2590392 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(o.title,3,false),test.`gram-tokens`('Transactions for Cooperative Environments',3,false),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
index 9353af4..b64edcc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`gram-tokens`(paper.title,3,false),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.sqlpp
index 992bf64..5f11c71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
select element o
from DBLP as o
with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'),0.500000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
index 168e1d3..5803200 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
select element {'Paper':paper_tokens,'Query':query_tokens}
from DBLP as paper
with paper_tokens as test.`word-tokens`(paper.title),
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
index 34e2503..bfd2e2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -54,7 +54,6 @@
create index msgNgramIx on TweetMessages (`message-text`:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
select element {'id':t2.tweetid,'topics':t2.`message-text`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.sqlpp
index cae1003..3baa6e9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
CSX as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.sqlpp
index 2ee4da7..c54814d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.sqlpp
@@ -50,7 +50,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
CSX as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.sqlpp
index a489041..63ed521 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
DBLP as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.sqlpp
index 2bc986c..2fe8d55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
select element {'title1':o1.title,'title2':o2.title}
from DBLP as o1,
CSX as o2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.sqlpp
index 48ad150..5990928 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.sqlpp
index c07a2f3..d010dfa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.sqlpp
index 82cff81..72e230d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.sqlpp
index c01b607..5cbffa5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index_CSX on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.sqlpp
index 3d1c52d..8409159 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.sqlpp
index 1c0104c..a0fb769 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-check_04.adm";
select element {'arec':a,'brec':b,'ed':ed[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.sqlpp
index aabfef6..0c55cb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance-contains.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.sqlpp
index 9d83f4b..d08abb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.sqlpp
index 0a6952c..d0346f6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.sqlpp
index e2ac1bb..c83953d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.sqlpp
index bcdebcb..4175705 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index_CSX on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.sqlpp
index f4d260c..1ce70c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.sqlpp
index 800cf0c..08b4838 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
select element {'arec':a,'brec':b,'ed':ed}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
index c1c1ecb..d4b9759 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
index 1f12373..2d7da8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
index 92eb730..a6014d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -40,7 +40,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
index f370d4f..f9f49f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.sqlpp
@@ -52,7 +52,6 @@
create index ngram_index_CSX on CSX (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
index 36a0d94..4454616 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.sqlpp
@@ -51,7 +51,6 @@
create index ngram_index on DBLP (authors:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `3`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
index ada10dc..368dd19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
index 597c77e..b1d98f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
index 74bb1a3..82f104e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
index e7d9c64..2172ab7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.sqlpp
index 611df16..5a3e527 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.sqlpp
index eabf59e..53e43f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.sqlpp
index 0f1ab7d..58abf94 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.sqlpp
index 813e661..c39a4f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.sqlpp
index 8715f04..804afa9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.sqlpp
index bf74a6f..9f7b1ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.sqlpp
index 6b88146..cecc578 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -53,7 +53,6 @@
create index ngram_index on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.sqlpp
index bbfc778..5d60a3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -42,7 +42,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.sqlpp
index c7b2921..c1c56af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -54,7 +54,6 @@
create index ngram_index_CSX on CSX (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.sqlpp
index f87cc2b..17836cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.sqlpp
@@ -43,7 +43,6 @@
create index ngram_index on DBLP (title:string?) type ngram (3) enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
index 2965d41..42964bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
index 1d50ad5..43515f5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
index 7e4a3b4..b345298 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
index e7fe3a4..4c7fbbc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index_CSX on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.5f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
index 85f6690..ff26138 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
select element {'t1':t1.tweetid,'t2':t2.tweetid,'sim':sim[1]}
from TweetMessages as t1,
TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.sqlpp
index 7f17dd2..dd7deaa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.sqlpp
index 7b7c77e..aa0df4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.sqlpp
index 2ab7191..7d33066 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.sqlpp
index d18ca91..4415e0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index_CSX on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.sqlpp
index 591a1fd..0537954 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc[1]}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.sqlpp
index f748ecc..b8a492e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.sqlpp
index a0595e7..1e874f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.sqlpp
@@ -51,7 +51,6 @@
create index keyword_index on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.sqlpp
index c00ccd3..683fe68 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.sqlpp
@@ -40,7 +40,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.sqlpp
index 0b72ffb..9202300 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.sqlpp
@@ -52,7 +52,6 @@
create index keyword_index_CSX on CSX (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
select element {'arec':a,'brec':b}
from DBLP as a,
CSX as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.sqlpp
index 61dd4ee..2f37358 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title:string?) type keyword enforced;
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
select element {'arec':a,'brec':b,'jacc':jacc}
from DBLP as a,
DBLP as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
index 944d04e..61af23b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
@@ -60,7 +60,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
index fdde3eb..f071424 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -60,7 +60,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.sqlpp
index 79af4c2..710e48f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.sqlpp
@@ -58,7 +58,6 @@
create index rtree_index on MyData1 (point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/index-join_rtree-spatial-intersect-point.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.sqlpp
index b41932d..a6b86d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -58,7 +58,6 @@
create index rtree_index on MyData2 (point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.sqlpp
index 634b355..9d3e455 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -44,7 +44,6 @@
create index rtree_index on MyData (point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
select element {'a':a,'b':b}
from MyData as a,
MyData as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.sqlpp
index 90da1f8..89882f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.sqlpp
@@ -48,7 +48,6 @@
create index rtree_index2 on MyData2 (point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.sqlpp
index 22b4f78..f7c1ab4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.sqlpp
@@ -46,7 +46,6 @@
create index rtree_index on MyData1 (point:point?) type rtree enforced;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orderby-desc-using-gby.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orderby-desc-using-gby.sqlpp
index 6d349c0..d10934d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orderby-desc-using-gby.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orderby-desc-using-gby.sqlpp
@@ -49,7 +49,6 @@
;
create dataset Customers(CustomerType) primary key cid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/gby-using-orderby-desc.adm";
select element {'name':name,'age':age}
from Customers as c
group by c.name as name, c.age as age
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
index fd54dcb..1566ad8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-aggreg.sqlpp
@@ -39,7 +39,6 @@
;
create dataset Orders(OrderType) primary key oid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/orders-aggreg.adm";
select element {'cid':cid,'ordpercust':`orders-aggreg`.strict_count(g),'totalcust':`orders-aggreg`.strict_sum((
select element i.o.total
from g as i
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-composite-index-search.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-composite-index-search.sqlpp
index 5d67268..50a1e51 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-composite-index-search.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-composite-index-search.sqlpp
@@ -40,7 +40,6 @@
create index idx_Custkey_Orderstatus on Orders (o_custkey,o_orderstatus) type btree;
-write output to asterix_nc1:"/tmp/index_search.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_orderstatus':o.o_orderstatus}
from Orders as o
where ((o.o_custkey = 40) and (o.o_orderstatus = 'P'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_01.sqlpp
index b010a1f..6318ba9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_01.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_02.sqlpp
index deb8b4c..10f2183 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive-open_02.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_01.sqlpp
index 5243efd..44f965d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_01.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_02.sqlpp
index fa77884..e7daca8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-conjunctive_02.sqlpp
@@ -46,7 +46,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
from Orders as o
where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-open.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-open.sqlpp
index 366d4b4..5c8d744 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-open.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search-open.sqlpp
@@ -44,7 +44,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where (o.o_custkey = 40)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search.sqlpp
index b7d0368..3021c33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/orders-index-search.sqlpp
@@ -44,7 +44,6 @@
create index idx_Orders_Custkey on Orders (o_custkey) type btree;
-write output to asterix_nc1:"/tmp/index_search.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where (o.o_custkey = 40)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search-open.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search-open.sqlpp
index 294cc85..ebeddae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search-open.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search-open.sqlpp
@@ -42,7 +42,6 @@
;
create dataset Orders(OrderType) primary key o_orderkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/prim_index_search.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where (o.o_orderkey = 34)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search.sqlpp
index 4663c89..44fa5fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/prim-idx-search.sqlpp
@@ -42,7 +42,6 @@
;
create dataset Orders(OrderType) primary key o_orderkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/prim_index_search.adm";
select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
from Orders as o
where (o.o_orderkey = 34)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/pull_select_above_eq_join.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/pull_select_above_eq_join.sqlpp
index b03b85b..e1ef5be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/pull_select_above_eq_join.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/pull_select_above_eq_join.sqlpp
@@ -47,7 +47,6 @@
create dataset Visitors(VisitorType) primary key vid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
select element {'uid':user.uid,'vid':visitor.vid}
from Users as user,
Visitors as visitor
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push-project-through-group.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push-project-through-group.sqlpp
index 7807359..ae9f845 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push-project-through-group.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push-project-through-group.sqlpp
@@ -38,7 +38,6 @@
;
create dataset DBLP(DBLPType) primary key id WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/fuzzyjoin_080.adm";
select element {'id':paperDBLP.id,'matches':matches}
from DBLP as paperDBLP
with matches as (
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push_limit.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push_limit.sqlpp
index 168c659..8fcc53b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push_limit.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/push_limit.sqlpp
@@ -42,7 +42,6 @@
;
create dataset Orders(OrderType) primary key o_orderkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/push_limit.adm";
select element {'price':o.o_totalprice,'date':o.o_orderdate}
from Orders as o
where (o.o_totalprice > 100)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
index db8d8cd..37e3837 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q01_pricing_summary_report_nt.sqlpp
@@ -47,7 +47,6 @@
load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
-write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
select element i.l.l_quantity
from g as i
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
index 90e7a13..14049ad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q03_shipping_priority.sqlpp
@@ -78,7 +78,6 @@
create dataset Customers(CustomerType) primary key c_custkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/q3_shipping_priority.adm";
select element {'l_orderkey':l_orderkey,'revenue':revenue,'o_orderdate':o_orderdate,'o_shippriority':o_shippriority}
from Customers as c,
Orders as o,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
index 70288b2..0edf911 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q05_local_supplier_volume.sqlpp
@@ -110,7 +110,6 @@
create dataset Regions(RegionType) primary key r_regionkey WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/q5_local_supplier.adm";
select element {'n_name':n_name,'revenue':revenue}
from Customers as c,
(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q1.sqlpp
index 791f0b9..94c2a6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q1.sqlpp
@@ -51,7 +51,6 @@
;
create dataset User(UserType) primary key name WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/q1.adm";
select element {'name':user.name}
from User as user
where some i in user.interests satisfies (i = 'movies')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q2.sqlpp
index f2413ef..f87f884 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/q2.sqlpp
@@ -56,7 +56,6 @@
;
create dataset Event(EventType) primary key name WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/q2.adm";
select element {'sig_name':sig_name,'total_count':sig_sponsorship_count,'chapter_breakdown':by_chapter}
from Event as event,
event.sponsoring_sigs as sponsor
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/record_access.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/record_access.sqlpp
index b48dd57..5e321f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/record_access.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/record_access.sqlpp
@@ -23,5 +23,4 @@
use test;
-write output to asterix_nc1:"/tmp/rec_access.adm";
{'a':2}.a;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/issue730.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/issue730.sqlpp
index 0966369..77ac587 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/issue730.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/issue730.sqlpp
@@ -53,7 +53,6 @@
create index twmSndLocIx on TweetMessages (sender_location) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_issue730.adm";
select element {'message':t1.tweetid,'nearby-message':(
select element t2.tweetid
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
index 66d3762..898e289 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
index a30f22c..cc0f899 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -61,7 +61,6 @@
create index msgTextIx on TweetMessages (`message-text`) type keyword;
-write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
from TweetMessages as t2
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/query-issue838.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/query-issue838.sqlpp
index 12c29b0..75e1b35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/query-issue838.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/query-issue838.sqlpp
@@ -50,7 +50,6 @@
create index locationIdx on TweetMessages (`sender-location`) type rtree;
-write output to asterix_nc1:"rttest/query-issue838.adm";
select element {'subscription-id':sub.`subscription-id`,'changeSet':1,'execution-time':twitter.`current-datetime`(),'message-text':text}
from TweetHistorySubscriptions as sub,
(
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_01.sqlpp
index 02805cd..d5b58b5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_01.sqlpp
@@ -47,7 +47,6 @@
create index rtree_index on MyData1 (point) type rtree;
-write output to asterix_nc1:"rttest/index-join_rtree-spatial-intersect-point.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_02.sqlpp
index f87832e..6bde012 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -47,7 +47,6 @@
create index rtree_index on MyData2 (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
select element {'a':a,'b':b}
from MyData1 as a,
MyData2 as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_03.sqlpp
index afe7a72..e60b46a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -45,7 +45,6 @@
create index rtree_index on MyData (point) type rtree;
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
select element {'a':a,'b':b}
from MyData as a,
MyData as b
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index-open.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index-open.sqlpp
index 0b86e27..7a08804 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index-open.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index-open.sqlpp
@@ -46,7 +46,6 @@
create index rtree_index_point on MyData (point) type rtree;
-write output to asterix_nc1:"rttest/index_rtree-secondary-index-open.adm";
select element {'id':o.id}
from MyData as o
where test.`spatial-intersect`(o.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index.sqlpp
index ff8bbee..3764782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/rtree-secondary-index.sqlpp
@@ -46,7 +46,6 @@
create index rtree_index_point on MyData (point) type rtree;
-write output to asterix_nc1:"rttest/index_rtree-secondary-index.adm";
select element {'id':o.id}
from MyData as o
where test.`spatial-intersect`(o.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.sqlpp
index 6382029..539d1c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_01.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.sqlpp
index 5e9721a..f633baa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_02.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.sqlpp
index d2f20cc..6d9d56a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_03.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.sqlpp
index 98fd885..6158457 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_04.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.sqlpp
index 71d76e0..61d88a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_05.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.sqlpp
index 0d10ba5..deb5874 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_06.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.sqlpp
index 2eaa38e..0423e25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_07.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.sqlpp
index e485c05..16f65b4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-let-to-edit-distance-check_08.adm";
select element ed
from DBLP as o
with ed as test.`edit-distance`(o.authors,'Michael J. Carey')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_01.sqlpp
index 983e401..739b38c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_01.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_01.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Michael J. Carey') <= 2)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_02.sqlpp
index 0a3239c..da3aa4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_02.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_02.adm";
select element o
from DBLP as o
where (2 >= test.`edit-distance`(o.authors,'Michael J. Carey'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_03.sqlpp
index 092ea01..b630699 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_03.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_03.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Michael J. Carey') < 3)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_04.sqlpp
index 33bb84a..349ce29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_04.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_04.adm";
select element o
from DBLP as o
where (3 > test.`edit-distance`(o.authors,'Michael J. Carey'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_05.sqlpp
index c2d9a4f..c2903bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_05.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_05.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Michael J. Carey') >= 2)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_06.sqlpp
index c1227f2..6cfe0c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_06.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_06.adm";
select element o
from DBLP as o
where (2 <= test.`edit-distance`(o.authors,'Michael J. Carey'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_07.sqlpp
index 0881c0d..f5cc75e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_07.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_07.adm";
select element o
from DBLP as o
where (test.`edit-distance`(o.authors,'Michael J. Carey') > 2)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_08.sqlpp
index d9f210a..4b35543 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/edit-distance-to-edit-distance-check_08.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_edit-distance-to-edit-distance-check_08.adm";
select element o
from DBLP as o
where (2 < test.`edit-distance`(o.authors,'Michael J. Carey'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-edit-distance-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-edit-distance-check.sqlpp
index 8e68f90..fed1e82 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-edit-distance-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-edit-distance-check.sqlpp
@@ -39,7 +39,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_fuzzyeq-to-edit-distance-check.adm";
set `simfunction` `edit-distance`;
set `simthreshold` `1`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-jaccard-check.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-jaccard-check.sqlpp
index c5ddbb9..2fb912e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-jaccard-check.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/fuzzyeq-to-jaccard-check.sqlpp
@@ -39,7 +39,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_fuzzyeq-to-jaccard-check.adm";
set `simfunction` `jaccard`;
set `simthreshold` `0.8f`;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_01.sqlpp
index 75ca2d5..8fa3f7d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_01.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_02.sqlpp
index b116890..1e1a0d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_02.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_03.sqlpp
index 3e30ef4..f3c0c96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_03.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_04.sqlpp
index 26eb0f7..5ee96f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_04.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_05.sqlpp
index 092703d..020c39b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_05.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_06.sqlpp
index 088e433..1adbe0d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_06.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_07.sqlpp
index c9822d4..a324bf9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_07.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_08.sqlpp
index cdee6a6..57af933 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-let-to-jaccard-check_08.sqlpp
@@ -41,7 +41,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-let-to-jaccard-check_01.adm";
select element jacc
from DBLP as paper
with jacc as test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_01.sqlpp
index b255f01..ec98480 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_01.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_01.adm";
select element paper
from DBLP as paper
where (test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')) >= 0.800000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_02.sqlpp
index 4bba4da..0284425 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_02.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_02.adm";
select element paper
from DBLP as paper
where (0.800000f <= test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_03.sqlpp
index de8c7d3..6a38dba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_03.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_03.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_02.adm";
select element paper
from DBLP as paper
where (test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')) > 0.800000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_04.sqlpp
index 8fc6675..1dd0854 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_04.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_04.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_03.adm";
select element paper
from DBLP as paper
where (0.800000f < test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_05.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_05.sqlpp
index e185c9c..3c15ba7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_05.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_05.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_05.adm";
select element paper
from DBLP as paper
where (test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')) <= 0.800000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_06.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_06.sqlpp
index 6f04ef8..4f88eca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_06.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_06.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_06.adm";
select element paper
from DBLP as paper
where (0.800000f >= test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_07.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_07.sqlpp
index 78da87a..655fa6b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_07.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_07.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_07.adm";
select element paper
from DBLP as paper
where (test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')) < 0.800000f)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_08.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_08.sqlpp
index 59a02fc..c51c849 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_08.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/similarity/jaccard-to-jaccard-check_08.sqlpp
@@ -40,7 +40,6 @@
create dataset DBLP(DBLPType) primary key id;
-write output to asterix_nc1:"rttest/similarity_jaccard-to-jaccard-check_08.adm";
select element paper
from DBLP as paper
where (0.800000f > test.`similarity-jaccard`(test.`word-tokens`(paper.title),test.`word-tokens`('Transactions for Cooperative Environments')))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/dont-skip-primary-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/dont-skip-primary-index.sqlpp
index 200ed64..5a3ec94 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/dont-skip-primary-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/dont-skip-primary-index.sqlpp
@@ -29,7 +29,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-primary-16.adm";
create type test.TestType as
{
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-ngram-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-ngram-index.sqlpp
index bcf5c40..046a4e3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-ngram-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-ngram-index.sqlpp
@@ -41,7 +41,6 @@
create index ngram_index on DBLP (title) type ngram (3);
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
select element o
from DBLP as o
where /*+ skip-index */ test.contains(o.title,'Multimedia')
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-rtree-secondary-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-rtree-secondary-index.sqlpp
index 58934cb..d0671f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-rtree-secondary-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-rtree-secondary-index.sqlpp
@@ -51,7 +51,6 @@
create index rtree_index_point on MyData (point) type rtree;
-write output to asterix_nc1:"rttest/index_rtree-secondary-index.adm";
select element {'id':o.id}
from MyData as o
where /*+ skip-index */ test.`spatial-intersect`(o.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index-2.sqlpp
index 6235842..3b8ac91 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index-2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index-2.sqlpp
@@ -28,7 +28,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index.sqlpp
index 3ea7139..094569c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-secondary-btree-index.sqlpp
@@ -28,7 +28,6 @@
use test;
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
create type test.TestType as
{
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-word-index.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-word-index.sqlpp
index fd4fe81..65203c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-word-index.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/skip-index/skip-word-index.sqlpp
@@ -41,7 +41,6 @@
create index keyword_index on DBLP (title) type keyword;
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
select element o
from DBLP as o
with jacc as /*+ skip-index */ test.`similarity-jaccard`(test.`word-tokens`(o.title),test.`word-tokens`('Transactions for Cooperative Environments'))
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/sort-cust.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/sort-cust.sqlpp
index 5ce4d75..97de7f3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/sort-cust.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/sort-cust.sqlpp
@@ -49,7 +49,6 @@
;
create dataset Customers(CustomerType) primary key cid WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"/tmp/custlimit.adm";
select element {'custname':c.name,'custage':c.age}
from Customers as c
order by c.age
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_01.sqlpp
index 6829a26..930bcff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_01.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_01.sqlpp
@@ -27,7 +27,6 @@
use test;
-write output to asterix_nc1:"rttest/unnest-to-join_01.adm";
select element y
from [1,2,3,4,5,6] as x,
[4,5,6,7,8,9] as y
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_02.sqlpp
index 599fbc2..bd8bea3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest-to-join_02.sqlpp
@@ -27,6 +27,5 @@
use test;
-write output to asterix_nc1:"rttest/unnest-to-join_02.adm";
select element some x in [1,2,3,4,5,6],
y in [4,5,6,7,8,9] satisfies (x = y);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest_list_in_subplan.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest_list_in_subplan.sqlpp
index e9a7228..e5cdcf2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest_list_in_subplan.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries_sqlpp/unnest_list_in_subplan.sqlpp
@@ -46,7 +46,6 @@
create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank WITH {"node-group":{"name":"group1"}};
-write output to asterix_nc1:"rttest/unnest_list_in_subplan.adm";
select element {'id':idDBLP,'tokens':tokensDBLP}
from DBLP as paperDBLP
with idDBLP as paperDBLP.id,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
new file mode 100644
index 0000000..e75ce49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
new file mode 100644
index 0000000..2c83436
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC), $$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
new file mode 100644
index 0000000..56db4c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-1/query3.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$75(ASC), $$68(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
new file mode 100644
index 0000000..99cd582
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
new file mode 100644
index 0000000..6864a95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
new file mode 100644
index 0000000..b4ee1b0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-2/query3.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$69(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
new file mode 100644
index 0000000..54e1f18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query1.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
new file mode 100644
index 0000000..0bc4272
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query2.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
new file mode 100644
index 0000000..6864a95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query3.plan
@@ -0,0 +1,29 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC), $$43(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
new file mode 100644
index 0000000..f251b77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-3/query4.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC), $$70(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
new file mode 100644
index 0000000..778f88d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query1.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$53(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
new file mode 100644
index 0000000..8db17af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query2.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$64(ASC), $$55(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
new file mode 100644
index 0000000..d28ee6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$61(ASC), $$53(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
new file mode 100644
index 0000000..22b9b1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/join-unnest-queries/use-case-4/query4.plan
@@ -0,0 +1,36 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$87(ASC), $$79(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN (TestYelp.YelpCheckinDateMarkers) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
new file mode 100644
index 0000000..2c5b278
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query1.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
new file mode 100644
index 0000000..761ed03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query2.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
new file mode 100644
index 0000000..f2819ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-1/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
new file mode 100644
index 0000000..e29b93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query1.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
new file mode 100644
index 0000000..9843256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query2.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
new file mode 100644
index 0000000..b9e4c18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-2/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$63(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
new file mode 100644
index 0000000..1d92106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query1.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
new file mode 100644
index 0000000..45549a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query2.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
new file mode 100644
index 0000000..f8aad44
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-3/query3.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
new file mode 100644
index 0000000..84dc8c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query1.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
new file mode 100644
index 0000000..a689c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query2.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$49(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
new file mode 100644
index 0000000..bfdc30a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/use-case-4/query3.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- SUBPLAN |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- SUBPLAN |LOCAL|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ASSIGN |LOCAL|
+ -- UNNEST |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
new file mode 100644
index 0000000..85a80b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-1/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
new file mode 100644
index 0000000..9af45a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-2/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
new file mode 100644
index 0000000..bb72552
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-3/query3.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
new file mode 100644
index 0000000..4cf8dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query1.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
new file mode 100644
index 0000000..4cb1efa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/use-case-4/query2.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..9177af3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-3-level-record-path/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
new file mode 100644
index 0000000..883fa06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query1.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$34(ASC), $$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
new file mode 100644
index 0000000..e4d4667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-pk/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC), $$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
new file mode 100644
index 0000000..2ad5dca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
new file mode 100644
index 0000000..afcd3a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-composite-sk/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$58(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
new file mode 100644
index 0000000..3a14007
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/closed/with-filter-fields/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
new file mode 100644
index 0000000..85a80b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$33(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
new file mode 100644
index 0000000..a65a2c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-1/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
new file mode 100644
index 0000000..9af45a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-2/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
new file mode 100644
index 0000000..3f9647b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$35(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
new file mode 100644
index 0000000..bb72552
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-3/query3.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
new file mode 100644
index 0000000..4cf8dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query1.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
new file mode 100644
index 0000000..4cb1efa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/use-case-4/query2.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$66(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
new file mode 100644
index 0000000..0202127
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query1.plan
@@ -0,0 +1,23 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
new file mode 100644
index 0000000..9177af3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-3-level-record-path/query2.plan
@@ -0,0 +1,26 @@
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ -- STREAM_PROJECT |UNPARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$60(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDates) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
new file mode 100644
index 0000000..8ae90bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/with-composite-sk/query1.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.YelpCheckin) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH (TestYelp.YelpCheckin.IdxYelpCheckinDatesTimes) |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
index d3c5a2c..d474d69 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
index 6a10149..363ae33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_01.ast
index 5b100e0..c7c9ec0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_01.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_02.ast
index 3de3b5e..1cf13e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_02.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_03.ast
index 9950cbb..f62139b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-join_03.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.ast
index 7650cff..0ab184c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_01.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.ast
index 3a38cb9..724c8b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_02.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.ast
index eb17fc3..88f961a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_03.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.ast
index 1958182..41bc6ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_04.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.ast
index c6c02b1..5328535 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_05.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.ast
index f3b85dd..01e2c9f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-composite-key-prefix-join_06.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[fname], [lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-multipred.ast
index dff4049..9047b4c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-multipred.ast
@@ -31,7 +31,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Orders(OrderType) partitioned by [[oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_01.ast
index 85ce2d6..9f2b831 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join-neg_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_02.ast
index db8b535..831a0a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join-neg_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join-neg_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_01.ast
index 9fbb8f1..9f2b831 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_02.ast
index dd2d5c0..831a0a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_03.ast
index 72ee37b..9c149c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_03.ast
@@ -31,7 +31,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Orders(OrderType) partitioned by [[oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_04.ast
index 2bbf27f..d967ba8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_04.ast
@@ -31,7 +31,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Orders(OrderType) partitioned by [[oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_05.ast
index c02a153..08bc286 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-equi-join_05.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_01.ast
index e99a722..846b36a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-ge-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_02.ast
index 968d0ce..0b93430 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-ge-join_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-ge-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_01.ast
index aaa16a4..a51fded 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-gt-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_02.ast
index 53c60ed..1569d25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-gt-join_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-gt-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_01.ast
index 7c6cd3d..02aa711 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-le-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_02.ast
index d97a727..6f1b1bf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-le-join_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-le-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_01.ast
index 9a305b6..c1e4a99 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_01.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-lt-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_02.ast
index ae4c9ed..2638b81 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/primary-lt-join_02.ast
@@ -8,7 +8,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[key1]]
DatasetDecl DsTwo(TestType) partitioned by [[key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-lt-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multiindex.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multiindex.ast
index 0ce7de3..5b48307 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multiindex.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multiindex.ast
@@ -33,7 +33,6 @@
]
DatasetDecl FacebookUsers(FacebookUserType) partitioned by [[id]]
DatasetDecl FacebookMessages(FacebookMessageType) partitioned by [[message-id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multiindex.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multipred.ast
index c5348a5..de170a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join-multipred.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_01.ast
index abf836a..ab3cac7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_02.ast
index 4ca8e3e..ab3cac7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_03.ast
index de6ec10..33c0c44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index-join/secondary-equi-join_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-01.ast
index a110598..17471bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-01.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-01.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-02.ast
index 8bd4191..e2c2783 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-02.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-02.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-03.ast
index b17aba4..78fd05e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-03.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-03.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-04.ast
index 0c5b0b8..99240d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-04.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-04.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-05.ast
index 6f2636a..bed0a4e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-05.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-05.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-06.ast
index 8f62e88..eb39bb7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-06.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-06.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-07.ast
index 79b8f81..7e9d89f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-07.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-07.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-08.ast
index 6186d45..e56818d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-08.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-08.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-09.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-09.ast
index b4d23a1..30d05df 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-09.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-09.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-09.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-10.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-10.ast
index ec74a73..a0f482e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-10.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-10.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-10.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-11.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-11.ast
index d77d307..7bebb27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-11.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-11.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-11.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-12.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-12.ast
index d32d25f..767f4e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-12.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-12.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-12.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-13.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-13.ast
index a88694a..786c79c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-13.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-13.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-13.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-14.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-14.ast
index fe8f3c2..f82c985 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-14.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-14.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-14.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-15.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-15.ast
index e0a006d..3910eee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-15.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-15.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-15.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-16.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-16.ast
index ec18343..c409193 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-16.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-16.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-16.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-17.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-17.ast
index c6fc17b..359ba26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-17.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-17.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-17.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-18.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-18.ast
index 291caac..fb3af80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-18.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-18.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-18.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-19.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-19.ast
index 3cdfc00..782601e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-19.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-19.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-19.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-20.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-20.ast
index e75f875..9fa09f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-20.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-20.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-20.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-21.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-21.ast
index 991ad2b..34bd3f4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-21.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-21.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-21.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-22.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-22.ast
index 9ae6cbf..ef0139a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-22.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-22.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-22.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-23.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-23.ast
index 6e59c20..b7788be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-23.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-23.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-23.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-24.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-24.ast
index f595f46..f505e50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-24.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-24.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-24.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-25.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-25.ast
index 1b35152..34fb938 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-25.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-25.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-25.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-26.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-26.ast
index 292ab52..900ec9e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-26.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-26.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-26.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-27.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-27.ast
index ea45444..1f40938 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-27.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-27.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-27.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-28.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-28.ast
index d136685..784f268 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-28.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-28.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-28.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-29.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-29.ast
index 8d12bf4..4f8161e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-29.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-29.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-29.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-30.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-30.ast
index 1d6ab85..ca3bbe4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-30.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-30.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-30.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-31.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-31.ast
index 2c1ac84..e5076ac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-31.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-31.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-31.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-32.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-32.ast
index 6ddf16a..d829dc4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-32.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-primary-32.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-32.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-33.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-33.ast
index 501128f..30228b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-33.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-33.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-33.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-34.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-34.ast
index 03744ba..3729451 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-34.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-34.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-34.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-35.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-35.ast
index bd3d2a6..166bcaf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-35.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-35.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-35.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-36.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-36.ast
index ae541cd..2020c60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-36.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-36.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-36.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-37.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-37.ast
index 9836775..0dc6f6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-37.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-37.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-37.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-38.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-38.ast
index def5c12..776acc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-38.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-38.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-38.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-39.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-39.ast
index 15aa892..0e9d9fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-39.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-39.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-39.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-40.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-40.ast
index bf3f516..c59949e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-40.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-40.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-40.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-41.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-41.ast
index 1917d14..6b50630 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-41.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-41.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-41.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-42.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-42.ast
index 18dc951..ca05e12 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-42.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-42.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-42.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-43.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-43.ast
index 4ab44cb..4fcbeac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-43.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-43.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-43.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-44.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-44.ast
index 7d3ec46..59976d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-44.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-44.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-44.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-45.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-45.ast
index 09aebbb..77a88c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-45.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-45.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-45.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-46.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-46.ast
index 19c187a..02fad35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-46.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-46.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-46.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-47.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-47.ast
index b0bd64d..35bac34 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-47.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-47.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-47.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-48.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-48.ast
index f7dd2f5..209cb66 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-48.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-48.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-48.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-49.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-49.ast
index 463d7b1..8ae2266 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-49.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-49.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-49.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-50.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-50.ast
index 5aa1c6d..594d0fd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-50.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-50.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-50.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-51.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-51.ast
index e5207d6..7df21f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-51.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-51.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-51.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-52.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-52.ast
index e49bdf3..28c473f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-52.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-52.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-52.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-53.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-53.ast
index 1b9bfb2..a2d8593 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-53.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-53.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-53.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-54.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-54.ast
index 75ad1ec..d407ed5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-54.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-54.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-54.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-55.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-55.ast
index 7b18229..01bb11f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-55.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-55.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-55.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-56.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-56.ast
index 94ef3cc..74cfe0f1b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-56.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-56.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-56.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-57.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-57.ast
index 7d2d833..bc86292 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-57.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-57.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-58.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-58.ast
index 7425220..bd7babb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-58.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-58.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-58.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-59.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-59.ast
index 9b272f2..fa4a469 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-59.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-59.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-59.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-60.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-60.ast
index d367499..d567695 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-60.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-60.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-60.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-61.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-61.ast
index 329f7a2..be2f9a7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-61.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-61.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-61.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-62.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-62.ast
index 52d7a34..1b2e2c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-62.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-62.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-62.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-63.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-63.ast
index 818e3ad..de5e251 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-63.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-63.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-63.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-64.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-64.ast
index 2f61ea8..dbf6129 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-64.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-64.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-64.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-65.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-65.ast
index a18469f..58add2a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-65.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-65.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-65.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-66.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-66.ast
index d4e56da..7ab913b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-66.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-66.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-66.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-67.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-67.ast
index 128db69..391b18a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-67.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/btree-index/btree-secondary-67.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-67.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/collocated.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/collocated.ast
index 712f8f0..a9f6ea2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/collocated.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/collocated.ast
@@ -17,7 +17,6 @@
]
DatasetDecl Users(UserType) partitioned by [[uid]]
DatasetDecl Visitors(VisitorType) partitioned by [[vid]]
-WriteOutputTo asterix_nc1:/tmp/fuzzy1.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-complex.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-complex.ast
index ba2ff41..96404b1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-complex.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-complex.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/consolidate-complex-selects.aql
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-simple.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-simple.ast
index 07b1760..fc9c1ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-simple.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/consolidate-selects-simple.ast
@@ -12,7 +12,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
-WriteOutputTo asterix_nc1:rttest/consolidate-selects-simple.aql
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/const-folding.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/const-folding.ast
index 557c08f..4891518 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/const-folding.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/const-folding.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/divide.adm
Query:
FieldAccessor [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/count-tweets.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/count-tweets.ast
index f7ec498..c225386 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/count-tweets.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/count-tweets.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl TwitterData(Tweet)is an external dataset
-WriteOutputTo asterix_nc1:/tmp/count-tweets.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/cust_group_no_agg.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/cust_group_no_agg.ast
index b02db45..02e5e32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/cust_group_no_agg.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/cust_group_no_agg.ast
@@ -12,7 +12,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
-WriteOutputTo asterix_nc1:/tmp/.adm
Query:
SELECT ELEMENT [
Variable [ Name=$name ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/denorm-cust-order.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/denorm-cust-order.ast
index d2085ab..aa01077 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/denorm-cust-order.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/denorm-cust-order.ast
@@ -31,7 +31,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Orders(OrderType) partitioned by [[oid]]
-WriteOutputTo asterix_nc1:/tmp/custorder.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/distinct_aggregate.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/distinct_aggregate.ast
index 410d63f..eac5e3e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/distinct_aggregate.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/distinct_aggregate.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl LineItems_q1(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
-WriteOutputTo asterix_nc1:rttest/tpch_q1_pricing_summary_report_nt.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/filter-nested.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/filter-nested.ast
index 0d434de..e0da327 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/filter-nested.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/filter-nested.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/filter-nested.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-dblp-csx.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-dblp-csx.ast
index 56c7eb1..41e26cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-dblp-csx.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-dblp-csx.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/fj-dblp-csx.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase1.ast
index f0107fa..6915903 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase1.ast
@@ -17,7 +17,6 @@
]
DatasetDecl Users(UserType) partitioned by [[uid]]
DatasetDecl Visitors(VisitorType) partitioned by [[vid]]
-WriteOutputTo asterix_nc1:/tmp/rares03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase2-with-hints.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase2-with-hints.ast
index 7175ec1..ff63588 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase2-with-hints.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/fj-phase2-with-hints.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP_fuzzyjoin_078(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/fuzzyjoin_078.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/hashjoin-with-unnest.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/hashjoin-with-unnest.ast
index 3075d61..05bfa13 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/hashjoin-with-unnest.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/hashjoin-with-unnest.ast
@@ -7,7 +7,6 @@
]
DatasetDecl t1(TestType) partitioned by [[id]]
DatasetDecl t2(TestType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/hahsjoin-with-unnest.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inline-funs.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inline-funs.ast
index 2d4312e..d526e6f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inline-funs.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inline-funs.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:/tmp/inline_funs.adm
FunctionDecl f1([]) {
OperatorExpr [
LiteralExpr [LONG] [1]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inlined_q18_large_volume_customer.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inlined_q18_large_volume_customer.ast
index 6205982..79da617 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inlined_q18_large_volume_customer.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inlined_q18_large_volume_customer.ast
@@ -47,7 +47,6 @@
DatasetDecl LineItems(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
-WriteOutputTo asterix_nc1:/tmp/inlined_q18_large_volume_customer.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/introhashpartitionmerge.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/introhashpartitionmerge.ast
index bf1b799..cd51c93 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/introhashpartitionmerge.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/introhashpartitionmerge.ast
@@ -6,7 +6,6 @@
}
]
DatasetDecl TOKENSRANKEDADM(TOKENSRANKEDADMType) partitioned by [[rank]]
-WriteOutputTo asterix_nc1:rttest/introhashpartitionmerge.adm
Query:
SELECT ELEMENT [
FieldAccessor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains-panic.ast
index 75e47cc..33a5272 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains-panic.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains.ast
index ccd9f32..128a04b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-contains.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.ast
index 473028d..2938a94 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check-panic.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check.ast
index 6248728..356ff3c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-check.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-panic.ast
index fdf46f9..ddfff15 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance-panic.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance.ast
index d8917a4..a8ae207 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-edit-distance.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
index 2a4a892..a6c654a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
index 6c7dd82..5d80834 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard-check.ast
index d4eabb8..86752d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard-check.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard.ast
index fc39ba9..c01bbaf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ngram-jaccard.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.ast
index 1626785..60bee55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check-panic.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check.ast
index b3de7ad..84fc04d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-check.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-panic.ast
index b9e055b..9f7e18f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance-panic.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance.ast
index fb69611..d614e3d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-edit-distance.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
index eeca2a9..031cbd4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.ast
index 64f857c..8d70557 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-fuzzyeq-jaccard.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard-check.ast
index 6906987..8df89b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard-check.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard.ast
index 61f30e3..15b2b5e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/olist-jaccard.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
index d1cd1fb..1aa4c9c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard-check.ast
index 192b9a9..2e43d3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard-check.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard.ast
index 3b734f9..f9e8207 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/ulist-jaccard.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-contains.ast
index 964c8ef..128a04b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-contains.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.ast
index 88f2e79..6b4bde7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-fuzzyeq-jaccard.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard-check.ast
index 6b2ef84..546d12d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard-check.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard.ast
index 13a5f3b..d5e68f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-basic/word-jaccard.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
index 50562cc..ffb132d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
index 295bc07..f3d43c5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
index 9908ea7..2938a94 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.ast
index 360b8fe..1117bb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-let.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.ast
index 4bd24fe..cf9cbfa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-substring.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
index 3cc550e..fc096a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-let.ast
index 10cc2e1..62e48a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-let.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.ast
index 2b7df8c..e20f30e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ngram-jaccard-check-multi-let.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.ast
index 1a98c4c..792cafe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let-panic.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let.ast
index 1125acc..a09d310 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-edit-distance-check-let.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-jaccard-check-let.ast
index f4a90e6..bd7d49a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/olist-jaccard-check-let.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ulist-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ulist-jaccard-check-let.ast
index b9cc80e..c8097ef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ulist-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/ulist-jaccard-check-let.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ulist-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-let.ast
index 9343f70..4823915 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-let.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.ast
index 6a2c11a..954eedb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-complex/word-jaccard-check-multi-let.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.ast
index 884b632..b20f119 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance-inline.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.ast
index 659a3c6..96f2735 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-edit-distance.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-edit-distance.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.ast
index c3dff99..83204d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.ast
index 880cf73..3c73927 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.ast
index 21e6d40..0f83207 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard-inline.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.ast
index a8a3b7c..f37832a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ngram-jaccard.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-jaccard.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.ast
index 1a68dec..6545fd5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance-inline.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.ast
index fc91fc2..5907e08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-edit-distance.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-edit-distance.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.ast
index df63e8a..b51672d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.ast
index ab7d128..cfbdc16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.ast
index 13d4c9c..e666af4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard-inline.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.ast
index acabdb3..96e42a4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/olist-jaccard.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-jaccard.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.ast
index bff6924..b76106c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ulist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.ast
index 6c78a5a..b8e0b12 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard-inline.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.ast
index 63fcf0a..d70d587 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/ulist-jaccard.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ulist-jaccard.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.ast
index 58e5337..2e7ecb9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_word-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.ast
index db614be..55b97ef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard-inline.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard.ast
index bd8e3aa..d820181 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join-noeqjoin/word-jaccard.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_word-jaccard.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/issue741.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/issue741.ast
index e06271a..b7ce344 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/issue741.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/issue741.ast
@@ -21,7 +21,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_issue741.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
index 2404dd0..7bd60e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
index 496e162..c428614 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-contains.ast
index 2f0b5fc..e4d9d35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-contains.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_01.ast
index 5e342d7..463aa6e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_02.ast
index 5a3ee14..463aa6e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_03.ast
index 9567f1b..3af02d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_04.ast
index 391303a..0ea6180 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-contains.ast
index b9277ef..2a82a32 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance-contains.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-contains.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_01.ast
index e9d5fe7..055e28a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_02.ast
index 690329c..055e28a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_03.ast
index 6d8ce54..4d01a35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_04.ast
index f34e41f..1af2d2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-edit-distance_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
index 08baa3b..78b52bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
index 08baa3b..78b52bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
index c416bc8..ae4317d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
index a784bd9..efeae01 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
index fa42c63..efeae01 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
index 0e9708d..09abd16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_01.ast
index 620a4ce..816e3fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_01.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_02.ast
index 8e4844a..816e3fb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_02.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_03.ast
index 41ca5dd..0668bbb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_03.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_04.ast
index d201069..556cd23 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard-check_04.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_01.ast
index 363ed04..4ea0e6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_01.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_02.ast
index 038e003..4ea0e6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_02.ast
@@ -20,7 +20,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_03.ast
index fe3316e..658f60b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_03.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_04.ast
index e70b58a..ef6ac14 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ngram-jaccard_04.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_01.ast
index 036b09f..21958fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_02.ast
index 483fdde..21958fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_03.ast
index a3caf5d..d9d355f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_04.ast
index 4ce8f78..ad081b7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance-check_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_01.ast
index 5dfbcc8..7780db7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_02.ast
index 828fdf0..7780db7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_03.ast
index 757d663..b61116a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_04.ast
index cb4f2ac..9be12f6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-edit-distance_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
index e5d4a73..76e1f7c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.ast
index ef71a93..76e1f7c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.ast
index 093e0b9..c18b795 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-edit-distance_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
index 20c1286..4c964a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.ast
index af2d31a..4c964a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.ast
index 7bfa769..2fccfa2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-fuzzyeq-jaccard_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_01.ast
index 0eeff59..a1b980e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_02.ast
index f968782..a1b980e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_03.ast
index 39927e1..cad9520 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_04.ast
index 21f585b..7d97505 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard-check_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_01.ast
index d03aaba..153df58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_02.ast
index eb3b7e8..153df58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_03.ast
index 0b8979a..65099ba 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_04.ast
index 3b84c14..5531407 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/olist-jaccard_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
index e2306db..274cc8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.ast
index 6f2107c..274cc8f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.ast
index 26547cc..5f20eb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-fuzzyeq-jaccard_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_01.ast
index 89a6e1c..569710b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_02.ast
index 5273305..569710b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_03.ast
index f99a6c2..6aa0bde 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_04.ast
index e1d293d..b224e10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard-check_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_01.ast
index 726f1ce..d3717ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_02.ast
index 77e63a2..d3717ea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_03.ast
index d64050c..87c92b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_03.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_04.ast
index b7158b5..306f8c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/ulist-jaccard_04.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.ast
index aeb5c2d..390c2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.ast
index 51c8a69..390c2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.ast
index 413fcf7..eb03572 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-fuzzyeq-jaccard_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.ast
index 0f36b7e..7769440 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check-after-btree-access.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_01.ast
index 3a30e84..df0012a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_02.ast
index d728851..df0012a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_03.ast
index fc7198f..5b6cc6c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_04.ast
index 49d2acb..0c25c0b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_01.ast
index 0252523..c7633b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_02.ast
index e67df1e..c7633b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_03.ast
index 58000c8..f3edffa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_04.ast
index ef04a85..e2188b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/inverted-index-join/word-jaccard_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_01.ast
index b0813fd..0da0974 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_01.ast
@@ -60,7 +60,6 @@
ps_comment : string
}
]
-WriteOutputTo asterix_nc1:/tmp/join-super-key_01.adm
DatasetDecl LineItems(LineItemType) partitioned by [[l_partkey], [l_linenumber]]
DatasetDecl PartSupp(PartSuppType) partitioned by [[ps_partkey], [ps_suppkey]]
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_02.ast
index 0b6b9b4..e6ccd58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/join-super-key_02.ast
@@ -60,7 +60,6 @@
ps_comment : string
}
]
-WriteOutputTo asterix_nc1:/tmp/join-super-key_01.adm
DatasetDecl LineItems(LineItemType) partitioned by [[l_partkey], [l_linenumber]]
DatasetDecl PartSupp(PartSuppType) partitioned by [[ps_partkey], [ps_suppkey]]
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/limit-issue353.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/limit-issue353.ast
index b0caf68..e8aa0e6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/limit-issue353.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/limit-issue353.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
-WriteOutputTo asterix_nc1:/tmp/push_limit.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_01.ast
index b012a70..f762b79 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_01.ast
@@ -60,7 +60,6 @@
ps_comment : string
}
]
-WriteOutputTo asterix_nc1:/tmp/loj-super-key_01.adm
DatasetDecl LineItems(LineItemType) partitioned by [[l_partkey], [l_linenumber]]
DatasetDecl PartSupp(PartSuppType) partitioned by [[ps_partkey], [ps_suppkey]]
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_02.ast
index deee859..94076d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/loj-super-key_02.ast
@@ -60,7 +60,6 @@
ps_comment : string
}
]
-WriteOutputTo asterix_nc1:/tmp/loj-super-key_01.adm
DatasetDecl LineItems(LineItemType) partitioned by [[l_partkey], [l_linenumber]]
DatasetDecl PartSupp(PartSuppType) partitioned by [[ps_partkey], [ps_suppkey]]
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
index 7db8a94..e99d32d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
index fb366dc..84de385 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.ast
index 67ea475..0cb243c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_01.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.ast
index b39858a..a5aeb7e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_02.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.ast
index 5d1eed5..92f177c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-join_03.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.ast
index 06f2e0e..61bf105 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_01.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.ast
index 85bc30b..90d017a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_02.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.ast
index 90f6a42..08dffc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_03.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.ast
index b11f064..f08f7b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_04.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.ast
index bd87136..02e5784 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_05.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.ast
index ecc055b..24b1b3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-composite-key-prefix-join_06.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[nested, fname], [nested, lname]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-composite-key-prefix-prefix-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.ast
index 52c73a7..2868850 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-multipred.ast
@@ -41,7 +41,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Orders(OrderType) partitioned by [[nested, oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.ast
index cab3285..666e2b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join-neg_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join-neg_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_01.ast
index 169ac13..666e2b6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_02.ast
index 41a0dff..afeaab9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_02.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_03.ast
index 726b20e..3a5cca1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_03.ast
@@ -41,7 +41,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Orders(OrderType) partitioned by [[nested, oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_04.ast
index ec580f5..32d41dd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_04.ast
@@ -41,7 +41,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Orders(OrderType) partitioned by [[nested, oid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_05.ast
index db6d9b4..2741b18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-equi-join_05.ast
@@ -25,7 +25,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-equi-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-ge-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-ge-join_01.ast
index 5c842dd..c83d901 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-ge-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-ge-join_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-ge-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-gt-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-gt-join_01.ast
index 4876514..b02c707 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-gt-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-gt-join_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-gt-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-le-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-le-join_01.ast
index 068153b..f8c1824 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-le-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-le-join_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-le-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-lt-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-lt-join_01.ast
index fdcaf32..1cd7ce4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-lt-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/primary-lt-join_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl DsOne(TestType) partitioned by [[nested, key1]]
DatasetDecl DsTwo(TestType) partitioned by [[nested, key1]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_primary-lt-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$x ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.ast
index 8ab6d60..4ec0822 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multiindex.ast
@@ -43,7 +43,6 @@
]
DatasetDecl FacebookUsers(FacebookUserType) partitioned by [[nested, id]]
DatasetDecl FacebookMessages(FacebookMessageType) partitioned by [[nested, message-id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multiindex.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.ast
index 77a210c..68b854b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join-multipred.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.ast
index 612efd5..b3f2bb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index-join/secondary-equi-join_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-01.ast
index 84a1e94..d680942 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-01.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-01.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-02.ast
index 9dedd70..0e08f0c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-02.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-02.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-03.ast
index 5c5d531..b3c2957 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-03.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-03.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-04.ast
index c4e0cc0..0c63e40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-04.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-04.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-05.ast
index b3dfab3..f10891f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-05.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-05.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-06.ast
index 1144ebc..a589888 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-06.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-06.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-07.ast
index ecd662b..89e69db 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-07.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-07.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-08.ast
index a296ae2..50e555d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-08.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-08.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-09.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-09.ast
index 2812011..3897f2f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-09.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-09.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-09.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-10.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-10.ast
index d1a8a1e..eaeda77 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-10.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-10.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-10.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-11.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-11.ast
index f208244..272af2a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-11.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-11.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-11.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-12.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-12.ast
index f486f6b..280420b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-12.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-12.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-12.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-13.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-13.ast
index 649c228..d142df7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-13.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-13.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-13.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-14.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-14.ast
index 0142b3e..f64a78f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-14.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-14.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-14.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-15.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-15.ast
index d696040..c25fe8c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-15.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-15.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-15.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-16.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-16.ast
index 5fb6602..6ce3f08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-16.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-16.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-16.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-17.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-17.ast
index ff50e2c..dfcbb9d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-17.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-17.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-17.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-18.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-18.ast
index e53bbe1..52613e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-18.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-18.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-18.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-19.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-19.ast
index b9c0155..dd3d515 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-19.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-19.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-19.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-20.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-20.ast
index e14bf42..ed62f55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-20.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-20.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-20.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-21.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-21.ast
index a12c25d..943c9d1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-21.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-21.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-21.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-22.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-22.ast
index 8398ea6..707c671 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-22.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-22.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-22.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-23.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-23.ast
index 0ded911..1dcc805 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-23.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-23.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-23.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-24.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-24.ast
index fc07439..9db683c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-24.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-24.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-24.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-25.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-25.ast
index 9f777ec..e3b57ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-25.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-25.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-25.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-26.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-26.ast
index 4a23d2b..de6f5fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-26.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-26.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-26.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-27.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-27.ast
index 6dd8fd5..52cfe52 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-27.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-27.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-27.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-28.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-28.ast
index c98f3e4..b561c35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-28.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-28.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-28.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-29.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-29.ast
index a70c092..3ddb4d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-29.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-29.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-29.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-30.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-30.ast
index 458b2f1..dc7daca 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-30.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-30.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-30.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-31.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-31.ast
index 0b2de22..73adf93 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-31.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-31.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-31.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-32.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-32.ast
index 4ba7769..bf38a43 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-32.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-primary-32.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-32.adm
TypeDecl TestTypetmp [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-33.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-33.ast
index 73d253c..f92048f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-33.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-33.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-31.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-34.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-34.ast
index 3d67725..b561164 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-34.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-34.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-32.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-35.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-35.ast
index 2d3461c..1ddaaf7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-35.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-35.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-33.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-36.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-36.ast
index 9c98570..66cb06c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-36.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-36.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-34.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-37.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-37.ast
index 409e808..6d60873 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-37.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-37.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-35.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-38.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-38.ast
index 55b9e60..342bbe8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-38.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-38.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-36.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-39.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-39.ast
index a687d78..ba4aceb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-39.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-39.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-37.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-40.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-40.ast
index 073930e..9421a28 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-40.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-40.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-38.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-41.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-41.ast
index cfd72ff..7bc0e9b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-41.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-41.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-39.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-42.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-42.ast
index ea845f4..3194cb5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-42.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-42.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-40.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-43.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-43.ast
index 5e27769..b31f315 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-43.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-43.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-41.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-44.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-44.ast
index a58d942..8124191 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-44.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-44.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-42.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-45.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-45.ast
index 5a91f05..355c04c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-45.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-45.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-43.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-46.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-46.ast
index b49041a..bbedc89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-46.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-46.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-44.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-47.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-47.ast
index aeecf14..03ca0f7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-47.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-47.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-45.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-48.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-48.ast
index c458ea5..9d6d60b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-48.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-48.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-46.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-49.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-49.ast
index 1b550a7..41b35a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-49.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-49.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-47.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-50.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-50.ast
index f539b41..4a40d3b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-50.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-50.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-48.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-51.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-51.ast
index 2db63f3..4c61300 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-51.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-51.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-49.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-52.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-52.ast
index afcfdd8..16015e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-52.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-52.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-50.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-53.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-53.ast
index 8a133f1..9c9b3d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-53.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-53.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-51.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-54.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-54.ast
index 3d231c7..4e5318b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-54.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-54.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-52.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-55.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-55.ast
index 9c30011..ccf66fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-55.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-55.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-53.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-56.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-56.ast
index 43a364d..04b87d6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-56.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-56.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-54.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-57.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-57.ast
index 1adf046..b7d886d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-57.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-57.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-55.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-58.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-58.ast
index 96c4300..9138d2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-58.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-58.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-56.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-59.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-59.ast
index a01ad08..0f2dcea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-59.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-59.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-60.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-60.ast
index 5c4a5b1..d62cece 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-60.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-60.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-58.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-61.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-61.ast
index d52e5d0..6bf4f8b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-61.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-61.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-61.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-62.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-62.ast
index d53b8f4..393ff03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-62.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-62.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-62.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-63.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-63.ast
index 9fa240e..e3101eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-63.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/btree-index/btree-secondary-63.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-63.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.ast
index 39e743a..f2f3874 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains-panic.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains.ast
index 8d823c8..40fb80d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-contains.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
index b31bc52..2a95ea8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.ast
index 0727633..92ccce4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-check.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.ast
index c711fee13..87099e3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance-panic.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.ast
index f4fe1b3..cc7962d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-edit-distance.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
index 8ebaf4a..db72942 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
index 3373f5a..7c41155e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.ast
index 13f6890..e08d6d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard-check.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.ast
index c581611..14a42d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ngram-jaccard.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.ast
index e7b386f..85e80a8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check-panic.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.ast
index 77a1e63..4ddaecb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-check.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.ast
index ec18fdf..3a1ab9f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance-panic.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.ast
index 5672136..e5fa961 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-edit-distance.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
index a3ea856..b63219f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.ast
index 98aab2b..8c0fd60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-fuzzyeq-jaccard.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.ast
index 8944f56..295a778 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard-check.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard.ast
index 2b3fd59..a6b2ae2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/olist-jaccard.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_olist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
index 586bb84..099635d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-fuzzyeq-jaccard.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.ast
index 951cc2c..b763be7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard-check.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.ast
index 33be47b..b81e7fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/ulist-jaccard.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ulist-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-contains.ast
index c319895..40fb80d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-contains.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
index b03e6c7..97b519c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.ast
index e8bf4a2..1f1fd20 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard-check.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard.ast
index 40574f5..b42fa63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-basic/word-jaccard.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
index 0818407..1764999 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
index fffd48a..10b393a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
index abd2b19..2a95ea8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.ast
index 9b45cff..00a9333 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-let.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
index 5fa1ef2..f59bf75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
index 2067c42..0c1c535 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.ast
index 36cd36e..86e36c9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-let.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
index 88c222c..15d3932 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.ast
index f1740e6..4281123 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.ast
index dfa394c..db5ba71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-edit-distance-check-let.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.ast
index 02ad60e..dd47d1f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/olist-jaccard-check-let.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_olist-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.ast
index 794d5cb..8543304 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/ulist-jaccard-check-let.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ulist-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$c ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.ast
index f66ecd2..391e6c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-let.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.ast
index 2881a88..2c6144e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-complex/word-jaccard-check-multi-let.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
index b502870..a6061c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
index 63c7261..96da2c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.ast
index 95723fc..dbea78f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.ast
index fb8882c..a403eea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-contains.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-contains.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.ast
index 201b5ac..bd3534d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance-inline.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.ast
index 59d970e..45c7533 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-edit-distance_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
index 5ec3806..0a7fd63 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
index 22333cf..d1a170b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
@@ -30,7 +30,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.ast
index cfe0ee8..3ab26c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-check_01.ast
@@ -30,7 +30,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.ast
index 6230626..f3aeb6d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard-inline.ast
@@ -15,7 +15,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.ast
index a168d88..5fdb216 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ngram-jaccard_01.ast
@@ -30,7 +30,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.ast
index 371a58b..77ade61 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.ast
index 9bdd0e8..4e9e94c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance-inline.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.ast
index 5b964fb..71716eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-edit-distance_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
index 5fa8fa3..abff3c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-edit-distance_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
index 2551463..de0ea20 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-fuzzyeq-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.ast
index ab9edac..42a126e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.ast
index e867a37..a1336ad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard-inline.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.ast
index 8fa6c8a..a199ef1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/olist-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_olist-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
index 66c07ed..ce18f39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-fuzzyeq-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.7f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.ast
index 19e40f9..bbc0b31 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.ast
index 3f13c3a..751d3ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard-inline.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.ast
index d3c444d..f67ad09 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/ulist-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl Customers(CustomerType) partitioned by [[nested, cid]]
DatasetDecl Customers2(CustomerType) partitioned by [[nested, cid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ulist-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
index 78d9cab..0d0d504 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
index 36d3927..7b2779b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.ast
index 3a22300..e4f9c3f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.ast
index 7518da7..565fceb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard-inline.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard_01.ast
index 8ab5b17..5d47d35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/inverted-index-join/word-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
index 8bde23b..5728e85 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
index c486ca1..22c1c45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
@@ -28,7 +28,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.ast
index 6e2f4e3..182475c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl MyData1(MyRecordNested) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index-join_rtree-spatial-intersect-point.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.ast
index 88ede17..182475c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl MyData1(MyRecordNested) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.ast
index 911ded3..6a66e26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-index/rtree-index-join/spatial-intersect-point_03.ast
@@ -17,7 +17,6 @@
}
]
DatasetDecl MyData(MyRecordNested) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
index 35c6c68..8209f44 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
index ee998a5..20b86f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
@@ -26,7 +26,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
index 5002b83..bfceb08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
index e6a964f..e04170d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
@@ -26,7 +26,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.ast
index 5c25aa2..9f2946e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_01.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.ast
index 3beae22..853ad79 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_02.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.ast
index 7fb9602..47331e3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-join_03.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.ast
index 6a701d4..c25ef52 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_01.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.ast
index 7f0a206..d27f16d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_02.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.ast
index e3572b4..404f232 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_03.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.ast
index 50f9012..3219cea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_04.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.ast
index 77e6f42..dc8aa42 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_05.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.ast
index 219246c..051014f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-composite-key-prefix-join_06.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Names(NameType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.ast
index 5bee77f..077d184 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multiindex.ast
@@ -42,7 +42,6 @@
]
DatasetDecl FacebookUsers(FacebookUserType) partitioned by [[nested, id]]
DatasetDecl FacebookMessages(FacebookMessageType) partitioned by [[nested, message-id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multiindex.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.ast
index 251ebaa..1ea879c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join-multipred.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.ast
index fbafbf0..7be9eea 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.ast
index bac276a..fe8101a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.ast
index b2ea711..1d53b38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_03.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.ast
index d610097..c25d239 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_04.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.ast
index d0366f2..1d53b38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index-join/secondary-equi-join_05.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-33.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-33.ast
index 29936b8..a00559a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-33.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-33.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-39.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-34.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-34.ast
index db20503..52bee04 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-34.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-34.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-32.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-35.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-35.ast
index 64d82c8..ddcd928 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-35.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-35.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-33.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-36.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-36.ast
index 1f746fc..e9290e4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-36.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-36.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-34.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-37.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-37.ast
index 63fe2e4..f89b53b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-37.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-37.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-35.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-38.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-38.ast
index c3d3466..e3f9558 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-38.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-38.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-36.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-39.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-39.ast
index 98aa61e..d06e4ee 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-39.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-39.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-37.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-40.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-40.ast
index 6884c92..ab1785f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-40.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-40.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-38.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-41.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-41.ast
index 85d03a7..fe72015 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-41.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-41.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-39.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-42.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-42.ast
index f5d81c6..c43378e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-42.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-42.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-40.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-43.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-43.ast
index 182df2d..4d111f1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-43.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-43.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-41.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-44.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-44.ast
index 23dab44..28df789 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-44.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-44.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-42.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-45.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-45.ast
index 5ef1bb3..b0f3ed5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-45.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-45.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-43.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-46.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-46.ast
index d022650..cfe167c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-46.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-46.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-44.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-47.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-47.ast
index 09f1f2d..0568f1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-47.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-47.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-45.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-48.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-48.ast
index ed0a1c4..13608c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-48.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-48.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-46.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-49.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-49.ast
index 0c52bc7..55c50d4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-49.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-49.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-47.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-50.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-50.ast
index 1a86db6..c208574 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-50.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-50.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-48.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-51.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-51.ast
index 392be20..48f6afd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-51.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-51.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-49.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-52.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-52.ast
index e4720f6..af5ca37 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-52.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-52.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-50.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-53.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-53.ast
index e32c66c..10537e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-53.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-53.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-51.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-54.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-54.ast
index a7c957c..e9719ac 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-54.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-54.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-52.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-55.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-55.ast
index 839f352..fe75d58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-55.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-55.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-53.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-56.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-56.ast
index 5617432..c1d58a0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-56.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-56.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-54.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-57.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-57.ast
index 2d2fa57..7f9d8fd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-57.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-57.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-55.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-58.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-58.ast
index d0d6966..42fbc5a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-58.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-58.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-56.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-59.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-59.ast
index 09f02a7..54a6cbf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-59.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-59.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-60.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-60.ast
index a8498e3..7564984 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-60.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-60.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-58.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-61.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-61.ast
index 84ec9cb..f9eadaa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-61.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-61.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-59.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-62.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-62.ast
index f970c2c..6c65ba1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-62.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-62.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-62.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-63.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-63.ast
index ce5782f..d56aea1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-63.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/btree-index/btree-secondary-63.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-63.adm
TypeDecl TestTypetmp [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.ast
index cab3477..1ca6429 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains-panic.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.ast
index dc27e08..bc634d21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-contains.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
index 95b82a5..01bab27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check-panic.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.ast
index 6f79953..2c7dca1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-check.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.ast
index 1d0e98c..97dd0bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance-panic.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.ast
index 622bc67..22248d3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-edit-distance.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
index cd76e6c..39afb1b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
index 4eac1f3..c603732 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.ast
index 4d23eee..0244bad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard-check.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.ast
index 6e8bcec..2adf0d8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/ngram-jaccard.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-contains.ast
index a05ab69..bc634d21 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-contains.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
index b508d79..36574b2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-fuzzyeq-jaccard.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.ast
index 2cdb0a8..f8ddf07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard-check.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.ast
index b4790d5..3bcde2e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-basic/word-jaccard.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
index 53aabf3..ef74a8b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
index 963578c..74f0934 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
index 6c21d1d..01bab27 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.ast
index e7c5cdc..26ca2d3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-let.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
index 49970e7..19670c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-substring.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
index f58dc37..f00e0c5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.ast
index 5ff1c93..da18e07 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-let.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
index e09c194..45698f6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/ngram-jaccard-check-multi-let.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.ast
index 518ccad..bfccbd5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-let.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.ast
index 0dddc3b..d2c9849 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-complex/word-jaccard-check-multi-let.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
index ad10e1a..5f86327 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.ast
index a91d675..22e139f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_01.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.ast
index 99f37fe..3a5ae5c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_02.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.ast
index 96cf45d..be15269 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.ast
index 19e5761..9f462de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-contains_04.ast
@@ -16,7 +16,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.ast
index bf45f51..2e563f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.ast
index f8d095b..66c443d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.ast
index 4e035d3..5536c1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.ast
index aa2b585..fefa6c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.ast
index aa2b585..fefa6c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-check_05.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.ast
index 481d88b..b9e6a77 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-contains.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-contains.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.ast
index b593eb1..aab24c4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance-inline.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-edit-distance-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.ast
index 658f36d..c9ff573 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.ast
index bad80a0..3a2fa60 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.ast
index 8d5c48f..807577a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.ast
index 63a7e24..2f730ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.ast
index 63a7e24..2f730ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-edit-distance_05.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
index dc33114..762b394 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
index a0d33fa..392dbf4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
index 9c00f66..1c2af92 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
index f76df1d..e7b9585 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
index f76df1d..e7b9585 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
index e78296e..741ae8d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
index 2c72b66..07b6fec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
index 28f0316..8520bd2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
index 7fa83e4..868ef1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.ast
index 9c108d3..af96521 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.ast
index bd600b8..8b5690d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_02.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.ast
index f3e0583..4344aab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_03.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.ast
index beb9f7f..363b9ec 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-check_04.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.ast
index b685fd2..d7ade70 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard-inline.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.ast
index 6ea9758..c9a934f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_01.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.ast
index 607ea6e..98dcd74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_02.ast
@@ -29,7 +29,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.ast
index da33640..40bb2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_03.ast
@@ -14,7 +14,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.ast
index d8fb211..aa76615 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/ngram-jaccard_04.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
index 957db1c..03fcd00 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.ast
index 7325128..346afa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.ast
index 8ae897f..e2d4ac9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.ast
index fb2de63..752d44a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
index 9240411..5460253 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.ast
index 81152b2..34f2e71 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.ast
index ab13a8d..cdb5ff4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.ast
index a11fccb..9ddc000 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.ast
index 6b51ee3..3635a75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-check_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.ast
index 31e1ed3..7abc582 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard-inline.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.ast
index 5f693a1..d7ab4e2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_01.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.ast
index dce9644..2784ad5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_02.ast
@@ -28,7 +28,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.ast
index 9b2b54a..b40155d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_03.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.ast
index 5b113fa..97cd5e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/inverted-index-join/word-jaccard_04.ast
@@ -27,7 +27,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[nested, id]]
DatasetDecl CSX(CSXType) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
index 0b31824..18735af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
index 76669c4..094edaa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
@@ -27,7 +27,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[nested, tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.ast
index 6e2f4e3..182475c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl MyData1(MyRecordNested) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index-join_rtree-spatial-intersect-point.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.ast
index 88ede17..182475c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl MyData1(MyRecordNested) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.ast
index 911ded3..6a66e26 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_03.ast
@@ -17,7 +17,6 @@
}
]
DatasetDecl MyData(MyRecordNested) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.ast
index 023d81c..609bf1b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl MyData1(MyRecordOpen) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecordOpen) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.ast
index 023d81c..609bf1b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested-open-index/rtree-index-join/spatial-intersect-point_05.ast
@@ -17,7 +17,6 @@
]
DatasetDecl MyData1(MyRecordOpen) partitioned by [[nested, id]]
DatasetDecl MyData2(MyRecordOpen) partitioned by [[nested, id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj2.ast
index 36dfee3..bc1bc2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj2.ast
@@ -47,7 +47,6 @@
DatasetDecl LineItems(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
-WriteOutputTo asterix_nc1:/tmp/nested_loj.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj3.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj3.ast
index 46ba97f..bba8d70 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj3.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/nested_loj3.ast
@@ -57,7 +57,6 @@
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
DatasetDecl PartSupp(PartSuppType) partitioned by [[ps_partkey], [ps_suppkey]]
-WriteOutputTo asterix_nc1:/tmp/nested_loj.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
index 57290ac..022343a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
index 353e552..93bcbbd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2.ast
@@ -21,7 +21,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
index 9de1874..015365f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
index 77e3668..b632fe2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2.ast
@@ -21,7 +21,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.ast
index 1ff24b1..66e00f8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_01.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.ast
index 9bb34db..60ea95b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_02.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.ast
index d26ba99..07d1260 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-join_03.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.ast
index 9fe3a66..296eeef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_01.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.ast
index 19e8601..6e639f2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_02.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.ast
index 75ae301..fcf1a0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_03.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.ast
index 4a304a4..9855e16 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_04.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.ast
index 715ef35..afbba7d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_05.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_05.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.ast
index dd530af..6c1cb4f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-composite-key-prefix-join_06.ast
@@ -5,7 +5,6 @@
}
]
DatasetDecl Names(Name) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_secondary-composite-key-prefix-prefix-join_06.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.ast
index 8dd07dc..c22df4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multiindex.ast
@@ -32,7 +32,6 @@
]
DatasetDecl FacebookUsers(FacebookUserType) partitioned by [[id]]
DatasetDecl FacebookMessages(FacebookMessageType) partitioned by [[message-id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multiindex.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.ast
index fbdade6..f6415d3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join-multipred.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join-multipred.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.ast
index a58bd4e..b263350 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.ast
index 6932d46..b7d93ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.ast
index 5e21138..1330f19 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.ast
index e99796f..6ba7587 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.ast
index e99796f..6ba7587 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index-join/secondary-equi-join_05.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/btree-index-join_title-secondary-equi-join_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-33.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-33.ast
index 04c5b61..ba35b33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-33.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-33.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-31.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-34.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-34.ast
index f189044..07caf2a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-34.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-34.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-32.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-35.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-35.ast
index 229d649..53f49d9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-35.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-35.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-33.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-36.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-36.ast
index e275ed9..3b4f305 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-36.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-36.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-34.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-37.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-37.ast
index 66829c4..c7ea507 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-37.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-37.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-35.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-38.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-38.ast
index 19c7a87..70957c0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-38.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-38.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-36.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-39.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-39.ast
index d76d661..afbd55b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-39.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-39.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-37.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-40.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-40.ast
index 60931bb..63349ef 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-40.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-40.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-38.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-41.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-41.ast
index 207cee8..aae866c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-41.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-41.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-39.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-42.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-42.ast
index 307c6b2..a8575b7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-42.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-42.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-40.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-43.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-43.ast
index 2e89de2..b3c17c6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-43.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-43.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-41.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-44.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-44.ast
index 820d851..b457451 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-44.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-44.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-42.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-45.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-45.ast
index ef28ba9..4c4c9c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-45.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-45.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-43.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-46.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-46.ast
index 1c2de53..054c5ad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-46.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-46.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-44.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-47.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-47.ast
index 3de42f4..eab573e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-47.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-47.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-45.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-48.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-48.ast
index 5da3296..2efebe6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-48.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-48.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-46.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-49.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-49.ast
index bcc76f3..6c974ad 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-49.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-49.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-47.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-50.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-50.ast
index edeb3a7..2e46e6a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-50.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-50.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-48.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-51.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-51.ast
index 44c1e9d..65a8d64 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-51.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-51.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-49.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-52.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-52.ast
index 04c4b38..f550a1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-52.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-52.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-50.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-53.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-53.ast
index 6746204..d2d3d38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-53.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-53.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-51.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-54.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-54.ast
index 624dffa..5b40c9f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-54.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-54.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-52.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-55.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-55.ast
index 1f0bd3b..3014b75 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-55.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-55.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-53.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-56.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-56.ast
index d48eee5..df0ed8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-56.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-56.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-54.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-57.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-57.ast
index 2236703..bc6ef14 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-57.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-57.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-55.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-58.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-58.ast
index 8671ba8..1efbe05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-58.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-58.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-56.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-59.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-59.ast
index 0f8393f..d6d177f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-59.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-59.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-60.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-60.ast
index fd6c3e9..a52fc1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-60.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-60.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-58.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-61.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-61.ast
index cb9ad21..058f8de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-61.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-61.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-49.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-62.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-62.ast
index 8165db0..46ba8ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-62.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-62.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-62.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-63.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-63.ast
index cc144f2..dafb109 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-63.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/btree-index/btree-secondary-63.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-63.adm
TypeDecl TestType [
open RecordType {
id : integer
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.ast
index bd5a2e7..71aa0bf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains-panic.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.ast
index 1d589dc..b9614fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-contains.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.ast
index e480eab..c49f309 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check-panic.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.ast
index bd20143..d58dae9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-check.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.ast
index b7f394f..e1f4d3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance-panic.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.ast
index eb7b1ab..ba7509a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-edit-distance.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-edit-distance.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
index 9c7aef5..11793b3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-edit-distance.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-edit-distance.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
index 89001af..702d557 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-fuzzyeq-jaccard.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.ast
index 4211435..ac4c691 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard-check.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.ast
index 9526803..1a23f29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/ngram-jaccard.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-contains.ast
index fe7b0e2..b9614fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-contains.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.ast
index 78a31e2..4e4feb3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-fuzzyeq-jaccard.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-fuzzyeq-jaccard.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.ast
index 31c0415..5eb8167 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard-check.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard-check.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.ast
index 7985db6..9283d84 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-basic/word-jaccard.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
index 51054c1..c1edf5c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
index d063d5f..c072006 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic-nopanic_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
index 69f204a..c49f309 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let-panic.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let-panic.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.ast
index 77f53c8..8900a83 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-let.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.ast
index 77154d5..2153301 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-substring.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-substring.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
index 2275853..0ef2b1a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-edit-distance-check-word-tokens.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-edit-distance-check-word-tokens.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.ast
index bc4fb5f..0cb8b83 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-let.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.ast
index aa93ac3..e9b7ca2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/ngram-jaccard-check-multi-let.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_ngram-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.ast
index 49d9157..6d2befc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-let.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-let.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.ast
index 6f506bb..cb7837a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-complex/word-jaccard-check-multi-let.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-complex_word-jaccard-check-multi-let.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
index 446d219..4bb0198 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.ast
index a91d675..22e139f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_01.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.ast
index 4bb674f..8c769ab 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_02.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.ast
index 96cf45d..be15269 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.ast
index 19e5761..9f462de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-contains_04.ast
@@ -16,7 +16,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-contains-04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.ast
index 052ccba..ad13f2c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.ast
index b16b062..445cb5d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.ast
index 64c187e..a0c64b0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.ast
index a8b47c7..0bbd6de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.ast
index a8b47c7..0bbd6de 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_05.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.ast
index c76b1d4..db1bed2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-check_inline_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.ast
index e74dded..0c57ac7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance-contains.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance-contains.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.ast
index 7a73244..937a291 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.ast
index 27a5d9d..ff084ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.ast
index a888ea2..018a7ff 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.ast
index 25eb7e1..5e3c8b7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.ast
index 25eb7e1..5e3c8b7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_05.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.ast
index 43bf4ce..1b10baa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-edit-distance_inline_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-edit-distance_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
index 694e4cc..a1a9c13 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
index 3d999c4..ec90b10 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
index 7f0e2f4..2d45105 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_03.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
index 2a57621..9a9c141 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
index 2a57621..9a9c141 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-edit-distance_05.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-edit-distance_01.adm
Set simfunction=edit-distance
Set simthreshold=3
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
index 825df9a..10511a0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
index 161a7c9..9b34a96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
index dab9992..b0d2f40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
index af199f5..155a088 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.ast
index f878e0d..c23b931 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.ast
index 0473451..fa9aa23 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.ast
index 7749d91..826e2d3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.ast
index 16ebed1..f24bf38 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.ast
index dfe5998..b12790a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.ast
index bc6f3e3..61362cd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_01.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.ast
index 87f9c21..bfb12c5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_02.ast
@@ -19,7 +19,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.ast
index 082647c..a887b97 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.ast
index 1e56d15..51ec91b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_04.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.ast
index e07e7a7..57f70d2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_ngram-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.ast
index c40a066..df96cf3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.ast
index d15ca81..b29c0c3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.ast
index ca7e2f6..df24529 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.ast
index 1ffc872..3a6a2a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm
Set simfunction=jaccard
Set simthreshold=0.5f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.ast
index d34a751..730d8be 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.ast
index ed3a5a8..ffe6005 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.ast
index 9da89f0..b48dad1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.ast
index 523861b..1a81d62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.ast
index df66e19..3a08301 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.ast
index 53707b4..bc424cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard-check_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.ast
index 065f46b..e8bcccb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_01.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.ast
index 58a3af9..7d44a50 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_02.ast
@@ -18,7 +18,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.ast
index 34684ca..dc5fb35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.ast
index 6c56a32..d3149f9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_04.ast
@@ -17,7 +17,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl CSX(CSXType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.ast
index d53274d..74fb715 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/inverted-index-join/word-jaccard_inline_03.ast
@@ -8,7 +8,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-join_word-jaccard_04.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
index db36dd9..a87742f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
index 0443877..1d7cfcf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
@@ -22,7 +22,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.ast
index d6c6deb..cf2bed6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_01.ast
@@ -24,7 +24,6 @@
]
DatasetDecl MyData1(MyRecordOpen) partitioned by [[id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index-join_rtree-spatial-intersect-point.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.ast
index 1c6eb1b..f9f7d25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_02.ast
@@ -24,7 +24,6 @@
]
DatasetDecl MyData1(MyRecord) partitioned by [[id]]
DatasetDecl MyData2(MyRecordOpen) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.ast
index e7ebb5a..1d68eb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_03.ast
@@ -11,7 +11,6 @@
}
]
DatasetDecl MyData(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.ast
index fdf1e55..2ebb4e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_04.ast
@@ -12,7 +12,6 @@
]
DatasetDecl MyData1(MyRecord) partitioned by [[id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.ast
index fdf1e55..2ebb4e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/open-index-enforced/rtree-index-join/spatial-intersect-point_05.ast
@@ -12,7 +12,6 @@
]
DatasetDecl MyData1(MyRecord) partitioned by [[id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orderby-desc-using-gby.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orderby-desc-using-gby.ast
index ca16111..aaf0842 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orderby-desc-using-gby.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orderby-desc-using-gby.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:rttest/gby-using-orderby-desc.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-aggreg.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-aggreg.ast
index 4d8ace4..7e7cf14 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-aggreg.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-aggreg.ast
@@ -10,7 +10,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[oid]]
-WriteOutputTo asterix_nc1:/tmp/orders-aggreg.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-composite-index-search.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-composite-index-search.ast
index c69f338..1bb8234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-composite-index-search.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-composite-index-search.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_01.ast
index 79333be..d0a8825 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_01.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search-conjunctive.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_02.ast
index 11c7e13..c8ed7cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive-open_02.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search-conjunctive.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_01.ast
index f4aece7..8e138c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_01.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search-conjunctive.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_02.ast
index 840ed37..5d9836d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-conjunctive_02.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search-conjunctive.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-open.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-open.ast
index f7f3fce..b8558cf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-open.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search-open.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search.ast
index da5b02b..d1cb9c1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/orders-index-search.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/index_search.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search-open.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search-open.ast
index 027c353..2a6a62b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search-open.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search-open.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/prim_index_search.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search.ast
index 4c02734..64b29ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/prim-idx-search.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/prim_index_search.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/pull_select_above_eq_join.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/pull_select_above_eq_join.ast
index c723a60..269cbf8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/pull_select_above_eq_join.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/pull_select_above_eq_join.ast
@@ -19,7 +19,6 @@
]
DatasetDecl Users(UserType) partitioned by [[uid]]
DatasetDecl Visitors(VisitorType) partitioned by [[vid]]
-WriteOutputTo asterix_nc1:/tmp/pull-select-above-eq-join.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push-project-through-group.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push-project-through-group.ast
index 0e349bbd..0ef8ae7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push-project-through-group.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push-project-through-group.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/fuzzyjoin_080.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push_limit.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push_limit.ast
index bfb413a..58dda6b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push_limit.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/push_limit.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
-WriteOutputTo asterix_nc1:/tmp/push_limit.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q01_pricing_summary_report_nt.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q01_pricing_summary_report_nt.ast
index dc175a7..45ddb18 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q01_pricing_summary_report_nt.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q01_pricing_summary_report_nt.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl LineItem(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
-WriteOutputTo asterix_nc1:rttest/tpch_q1_pricing_summary_report_nt.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q03_shipping_priority.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q03_shipping_priority.ast
index c08dc5c..4732488 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q03_shipping_priority.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q03_shipping_priority.ast
@@ -47,7 +47,6 @@
DatasetDecl LineItems(LineItemType) partitioned by [[l_orderkey], [l_linenumber]]
DatasetDecl Orders(OrderType) partitioned by [[o_orderkey]]
DatasetDecl Customers(CustomerType) partitioned by [[c_custkey]]
-WriteOutputTo asterix_nc1:/tmp/q3_shipping_priority.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q05_local_supplier_volume.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q05_local_supplier_volume.ast
index 448c433..5e642a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q05_local_supplier_volume.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q05_local_supplier_volume.ast
@@ -76,7 +76,6 @@
DatasetDecl Suppliers(SupplierType) partitioned by [[s_suppkey]]
DatasetDecl Nations(NationType) partitioned by [[n_nationkey]]
DatasetDecl Regions(RegionType) partitioned by [[r_regionkey]]
-WriteOutputTo asterix_nc1:/tmp/q5_local_supplier.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q1.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q1.ast
index 4e8ad75..8398ecd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q1.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q1.ast
@@ -24,7 +24,6 @@
}
]
DatasetDecl User(UserType) partitioned by [[name]]
-WriteOutputTo asterix_nc1:/tmp/q1.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q2.ast
index 40229c3..1825acf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/q2.ast
@@ -30,7 +30,6 @@
}
]
DatasetDecl Event(EventType) partitioned by [[name]]
-WriteOutputTo asterix_nc1:/tmp/q2.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/record_access.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/record_access.ast
index af9e9be..cb5306c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/record_access.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/record_access.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:/tmp/rec_access.adm
Query:
FieldAccessor [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/issue730.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/issue730.ast
index 6f17b8f..ea955a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/issue730.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/issue730.ast
@@ -21,7 +21,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_issue730.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
index 0e3e5a6..12d3541 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
index c586622..5cdaa47 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.ast
@@ -23,7 +23,6 @@
}
]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/query-issue838.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/query-issue838.ast
index dfd39ff..56ba58b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/query-issue838.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/query-issue838.ast
@@ -14,7 +14,6 @@
]
DatasetDecl TweetHistorySubscriptions(TweetHistorySubscription) partitioned by [[subscription-id]]
DatasetDecl TweetMessages(TweetMessageType) partitioned by [[tweetid]]
-WriteOutputTo asterix_nc1:rttest/query-issue838.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_01.ast
index 2391565..7edf38f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_01.ast
@@ -13,7 +13,6 @@
]
DatasetDecl MyData1(MyRecord) partitioned by [[id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index-join_rtree-spatial-intersect-point.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_02.ast
index b29fe1f..7edf38f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_02.ast
@@ -13,7 +13,6 @@
]
DatasetDecl MyData1(MyRecord) partitioned by [[id]]
DatasetDecl MyData2(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_02.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_03.ast
index f04a100..d823e80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-index-join/spatial-intersect-point_03.ast
@@ -12,7 +12,6 @@
}
]
DatasetDecl MyData(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/rtree-index-join_spatial-intersect-point_03.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index-open.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index-open.ast
index b5d3bf1..d50fff2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index-open.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index-open.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl MyData(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index_rtree-secondary-index-open.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index.ast
index 4125b9a..a41eb9e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/rtree-secondary-index.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl MyData(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index_rtree-secondary-index.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.ast
index fe90f5c..e50d053 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_01.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.ast
index 877bcbd..65033b9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_02.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.ast
index 2fe2205..c9129aa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_03.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.ast
index 4d81225..5455e06 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_04.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.ast
index 9524164..abecec2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_05.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_05.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.ast
index 65642c2..0781ab0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_06.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_06.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.ast
index ee2107d..4d8112b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_07.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_07.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.ast
index 38e22b1..6690ae4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-let-to-edit-distance-check_08.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-let-to-edit-distance-check_08.adm
Query:
SELECT ELEMENT [
Variable [ Name=$ed ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_01.ast
index 1992919..6411be4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_01.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_02.ast
index fe12d67..69f00ed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_02.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_03.ast
index 58c443c..7efcf40 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_03.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_04.ast
index 6edbf57..05a816a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_04.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_05.ast
index 68b6427..f9528fc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_05.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_05.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_06.ast
index 42feccb..8b6e1f6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_06.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_06.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_07.ast
index c864e3c..e429260 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_07.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_07.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_08.ast
index 97ab3e7..00c167f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/edit-distance-to-edit-distance-check_08.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_edit-distance-to-edit-distance-check_08.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-edit-distance-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-edit-distance-check.ast
index b498f55..c0bb9c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-edit-distance-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-edit-distance-check.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_fuzzyeq-to-edit-distance-check.adm
Set simfunction=edit-distance
Set simthreshold=1
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-jaccard-check.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-jaccard-check.ast
index c000fe6..230cadf 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-jaccard-check.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/fuzzyeq-to-jaccard-check.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_fuzzyeq-to-jaccard-check.adm
Set simfunction=jaccard
Set simthreshold=0.8f
Query:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_01.ast
index 73d96c8..1e82b2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_01.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_02.ast
index e4e045b..9663372 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_02.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_03.ast
index b2d48ce..d8533cc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_04.ast
index 9be8f7e..8553de3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_05.ast
index 94af050..eb04b3a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_05.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_06.ast
index f29c220..3e0da72 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_06.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_07.ast
index b221064..13825a1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_07.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_08.ast
index d877c2c..bd35990 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-let-to-jaccard-check_08.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-let-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$jacc ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_01.ast
index edf7b0a..30af3d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_01.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_02.ast
index 8b23c9b..3a45e03 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_02.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_03.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_03.ast
index 456ddc3..0ae7b56 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_03.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_03.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_02.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_04.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_04.ast
index e36a6e7..d860c37 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_04.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_04.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_03.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_05.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_05.ast
index 8dbd31c..9fa77bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_05.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_05.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_05.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_06.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_06.ast
index a2f20e4..ce9f2c2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_06.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_06.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_06.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_07.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_07.ast
index 3a17439..93d198d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_07.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_07.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_07.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_08.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_08.ast
index e19f053..214fc62 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_08.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/similarity/jaccard-to-jaccard-check_08.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/similarity_jaccard-to-jaccard-check_08.adm
Query:
SELECT ELEMENT [
Variable [ Name=$paper ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/dont-skip-primary-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/dont-skip-primary-index.ast
index ec18343..c409193 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/dont-skip-primary-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/dont-skip-primary-index.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-primary-16.adm
TypeDecl TestType [
open RecordType {
fname : string,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-ngram-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-ngram-index.ast
index ccd9f32..128a04b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-ngram-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-ngram-index.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_ngram-contains.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-rtree-secondary-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-rtree-secondary-index.ast
index 4125b9a..a41eb9e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-rtree-secondary-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-rtree-secondary-index.ast
@@ -13,7 +13,6 @@
}
]
DatasetDecl MyData(MyRecord) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/index_rtree-secondary-index.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index-2.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index-2.ast
index 89f3acb..fa4a469 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index-2.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index-2.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index.ast
index 89f3acb..fa4a469 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-secondary-btree-index.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/btree-index_btree-secondary-57.adm
TypeDecl TestType [
open RecordType {
id : integer,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-word-index.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-word-index.ast
index 6ae94e1..59447ae 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-word-index.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/skip-index/skip-word-index.ast
@@ -9,7 +9,6 @@
}
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
-WriteOutputTo asterix_nc1:rttest/inverted-index-basic_word-jaccard.adm
Query:
SELECT ELEMENT [
Variable [ Name=$o ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/sort-cust.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/sort-cust.ast
index de97209..5c0b397 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/sort-cust.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/sort-cust.ast
@@ -20,7 +20,6 @@
}
]
DatasetDecl Customers(CustomerType) partitioned by [[cid]]
-WriteOutputTo asterix_nc1:/tmp/custlimit.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_01.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_01.ast
index d37b88e..e8ceb92 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_01.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_01.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/unnest-to-join_01.adm
Query:
SELECT ELEMENT [
Variable [ Name=$y ]
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_02.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_02.ast
index aedd68f..9192518 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_02.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest-to-join_02.ast
@@ -1,5 +1,4 @@
DataverseUse test
-WriteOutputTo asterix_nc1:rttest/unnest-to-join_02.adm
Query:
SELECT ELEMENT [
QuantifiedExpression SOME [
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest_list_in_subplan.ast b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest_list_in_subplan.ast
index b9fd2a2..477d221 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest_list_in_subplan.ast
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results_parser_sqlpp/unnest_list_in_subplan.ast
@@ -16,7 +16,6 @@
]
DatasetDecl DBLP(DBLPType) partitioned by [[id]]
DatasetDecl TOKENSRANKEDADM(TOKENSRANKEDADMType) partitioned by [[rank]]
-WriteOutputTo asterix_nc1:rttest/unnest_list_in_subplan.adm
Query:
SELECT ELEMENT [
RecordConstructor [
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..3e2eeda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessIDDates ON YelpCheckin (business_id, UNNEST dates);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..f911125
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..ecef50a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..3fd8b82
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-2.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..abd1c6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..371bc27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-3.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..626e797
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..5f7417c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-4.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..47b4d5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..277d30b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-3-level-record-path.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..415054c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation on array indexes with more than one PK.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..38b50e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..1de5989
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array index with filter fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..38b50e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/with-composite-pk.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..aa3a456
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..f911125
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+LOAD DATASET YelpCheckin
+USING localfs (("path"="asterix_nc1://data/yelp-checkin/use-case-1.json"),
+ ("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..5241534
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..dbf055c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusinessID ON YelpCheckin (UNNEST dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id, UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..0b3cfea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0452c4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+
+/* It would be more user-friendly to use (UNNEST checkin_times.dates), (business_id), but we are able to infer this
+ from our grammar alone. We leave this in to also exercise our parser. */
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST checkin_times.dates, business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d820ff8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
new file mode 100644
index 0000000..ad50985
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.5.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DROP INDEX YelpCheckin.IdxYelpCheckinDates;
+CREATE INDEX IdxYelpCheckinDatesTimes ON YelpCheckin (UNNEST checkin_times SELECT date, time);
+// CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin ((UNNEST checkin_times SELECT date, time), (business_id));
+// CREATE INDEX IdxYelpCheckinBusinessDatesTimes ON YelpCheckin (business_id, UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..f42b348
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..f571dbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+// CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin ((UNNEST checkin_times UNNEST dates), business_id);
+// CREATE INDEX IdxYelpCheckinBusinessDates ON YelpCheckin (business_id, UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..acd005f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..5ef37bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..88ac065
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation on array indexes with more than one PK.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..29fd915
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array index with filter fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..4076942
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..0d37224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..01511bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..abacfe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..e2340e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->atomic index, as well as a composite index on both fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..9594cf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
new file mode 100644
index 0000000..982affc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.5.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DROP INDEX YelpCheckin.IdxYelpCheckinDates;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..27f01aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT
+WHERE "2016-04-26" = CT.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..3487452
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the bulk-loading operation for an array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..c590ecb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..8ba650f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
new file mode 100644
index 0000000..c26bf92
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-mixed-composite/index-mixed-composite.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create a mixed composite atomic and array index (not supported for now).
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS {
+ time: string
+};
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinTimeBusiness ON YelpCheckin ((UNNEST checkin_times SELECT time), (business_id));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
new file mode 100644
index 0000000..685485b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-on-closed-array/index-on-closed-array.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create a typed index on a non-existent closed field within an array.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS CLOSED {
+ time: string
+};
+CREATE TYPE CheckinType AS CLOSED {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..83d5407
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are not able to index more than one array field at different depths.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string],
+ times: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates, UNNEST times);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
new file mode 100644
index 0000000..d610c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-two-array-fields/index-two-array-fields.2.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin ((UNNEST checkin_times SELECT date), UNNEST checkin_times UNNEST times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
new file mode 100644
index 0000000..baf5bc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/index-with-enforced-type/index-with-enforced-type.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that an error is thrown when trying to create an enforced typed index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinTimeType AS {
+ time: string
+};
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [CheckinTimeType]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string) ENFORCED;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
new file mode 100644
index 0000000..2402382
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that a specified path that is valid through record-pathing, but invalid through array pathing.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_time: {
+ date: string,
+ time: string
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time SELECT date);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
new file mode 100644
index 0000000..d658dbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/error-handling/invalid-array-path/invalid-array-path.2.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that a specified path that is valid through record-pathing, but invalid through array pathing.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_time: {
+ date: string,
+ time: string
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_time.date);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..7498c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..ba9a31b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": []
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": []
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..81b322c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp
new file mode 100644
index 0000000..9467e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.6.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d65a2eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..4fd7c64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp
new file mode 100644
index 0000000..b3f8466
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..590c9d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..0360912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp
new file mode 100644
index 0000000..b5b6c72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.6.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
new file mode 100644
index 0000000..1697fb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index, with an additional atomic index:
+ * 1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: int,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
new file mode 100644
index 0000000..8ee1570
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": []
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": []
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
new file mode 100644
index 0000000..965ccc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..6ff0c88
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for a composite array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: int,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..2f7a9fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp
new file mode 100644
index 0000000..5ec7879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..c6739ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index having a dataset filter field:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id WITH FILTER ON business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..ab374bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..ba9a31b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": []
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": []
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp
new file mode 100644
index 0000000..055fe8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..ef8d1c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times.times : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp
new file mode 100644
index 0000000..9467e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.6.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..a440550
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..768692b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times SELECT time : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..4fd7c64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp
new file mode 100644
index 0000000..b3f8466
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.6.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..2757b02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..068482f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->record->array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ business_id: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY business_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinTimes ON YelpCheckin (UNNEST checkin_times UNNEST times : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..0360912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp
new file mode 100644
index 0000000..b5b6c72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.6.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..3406a35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
new file mode 100644
index 0000000..cc59e2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for an array->atomic index, with an additional atomic index:
+ * 1) Insert into an empty index (non bulk-load operation), additionally with two records that have no array-index qualifying entries.
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all *original* (all records have qualifying array-index entries now) entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: int
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
new file mode 100644
index 0000000..8ee1570
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.update.sqlpp
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": []
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": []
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
new file mode 100644
index 0000000..965ccc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
new file mode 100644
index 0000000..69fe7cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.7.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..da9eb7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the following DML actions for a composite array->atomic index:
+ * 1) Insert into an empty index (non bulk-load operation).
+ * 2) Delete all-but-one entry from the index.
+ * 3) Upsert all entries into the index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: int
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..2f7a9fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp
new file mode 100644
index 0000000..ea1dba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.4.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+DELETE FROM YelpCheckin C
+WHERE C.business_id != "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp
new file mode 100644
index 0000000..5ec7879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.6.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+UPSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp
new file mode 100644
index 0000000..baf548c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > " " AND
+ D.time > " " AND
+ C.business_id > " ";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..39f18a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ * 1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
+ * 2) Implicit INNER JOIN query, with additional predicate on the outer collection.
+ * 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..bb9eea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp
new file mode 100644
index 0000000..a66da8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp
new file mode 100644
index 0000000..8bdf70e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-1/use-case-1.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..a00ffca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an record->array->atomic index.
+ * 1) INNER JOIN query, with additional predicate on inner collection before the unnest.
+ * 2) INNER JOIN query, with additional predicate on the outer collection.
+ * 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..93777ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp
new file mode 100644
index 0000000..dba4085
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp
new file mode 100644
index 0000000..c5598d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-2/use-case-2.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..fed0346
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->record->atomic index.
+ * 1) INNER JOIN query, with additional predicate on inner collection before the unnest.
+ * 2) INNER JOIN query, with additional predicate on inner collection after the unnest.
+ * 3) INNER JOIN query, with additional predicate on the outer collection.
+ * 4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ * TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..147f99b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..5a380e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp
new file mode 100644
index 0000000..eb7687f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times D
+WHERE D.date /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp
new file mode 100644
index 0000000..2719057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-3/use-case-3.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D.date
+ FROM YelpCheckin C, C.checkin_times D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..41d36e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->record->array->atomic index.
+ * 1) INNER JOIN query, with additional predicate on inner collection before the first unnest.
+ * 2) INNER JOIN query, with additional predicate on inner collection before the first unnest and before the final unnest.
+ * 3) INNER JOIN query, with additional predicate on the outer collection.
+ * 4) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ * TODO (GLENN) 5) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ checkin_times_id: int,
+ dates: [string],
+ times: [string]
+ }]
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..21105b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,487 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "checkin_times_id": 4,
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "checkin_times_id": 4,
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "checkin_times_id": 4,
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "checkin_times_id": 1,
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "checkin_times_id": 2,
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "checkin_times_id": 3,
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "checkin_times_id": 4,
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "checkin_times_id": 5,
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..7220d42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp
new file mode 100644
index 0000000..cc884df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND CT.checkin_times_id = 1
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..7e6c12b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D /*+ indexnl */ = M.date
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp
new file mode 100644
index 0000000..5f6ed5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/use-case-4/use-case-4.8.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+) AS CD ON CD /*+ indexnl */ = M.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..039df8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ * 1) Implicit INNER JOIN query, with additional predicate on inner collection before the unnest.
+ * 2) Implicit INNER JOIN query, with additional predicate on the outer collection.
+ * 3) Explicit INNER JOIN query using UNNEST subquery, no additional predicate.
+ * TODO (GLENN) 4) Explicit LEFT OUTER JOIN query using UNNEST subquery, no additional predicate.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+CREATE TYPE CheckinDatesToMarkersType AS {
+ marker: string,
+ datetime: string,
+ date: string,
+ time: string
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE DATASET YelpCheckinDateMarkers (CheckinDatesToMarkersType) PRIMARY KEY marker;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp
new file mode 100644
index 0000000..c1eca5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.3.update.sqlpp
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckinDateMarkers (
+ [
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5", "datetime": "2016-04-26 19:49:16", "date": "2016-04-26", "time": "19:49:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f6", "datetime": "2016-08-30 18:36:57", "date": "2016-08-30", "time": "18:36:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f7", "datetime": "2016-10-15 02:45:18", "date": "2016-10-15", "time": "02:45:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f8", "datetime": "2016-11-18 01:54:50", "date": "2016-11-18", "time": "01:54:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f9", "datetime": "2017-04-20 18:39:06", "date": "2017-04-20", "time": "18:39:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fa", "datetime": "2017-05-03 17:58:02", "date": "2017-05-03", "time": "17:58:02" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fb", "datetime": "2019-03-19 22:04:48", "date": "2019-03-19", "time": "22:04:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fc", "datetime": "2018-05-25 19:52:07", "date": "2018-05-25", "time": "19:52:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fd", "datetime": "2018-09-18 16:09:44", "date": "2018-09-18", "time": "16:09:44" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882fe", "datetime": "2019-10-18 21:29:09", "date": "2019-10-18", "time": "21:29:09" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c36882ff", "datetime": "2011-05-03 20:54:05", "date": "2011-05-03", "time": "20:54:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688300", "datetime": "2011-08-23 20:49:45", "date": "2011-08-23", "time": "20:49:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688301", "datetime": "2014-12-04 06:13:01", "date": "2014-12-04", "time": "06:13:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688302", "datetime": "2016-11-16 19:25:55", "date": "2016-11-16", "time": "19:25:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688303", "datetime": "2016-06-18 21:35:45", "date": "2016-06-18", "time": "21:35:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688304", "datetime": "2016-10-15 18:17:51", "date": "2016-10-15", "time": "18:17:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688305", "datetime": "2014-08-27 17:49:18", "date": "2014-08-27", "time": "17:49:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688306", "datetime": "2015-12-19 21:30:31", "date": "2015-12-19", "time": "21:30:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688307", "datetime": "2018-11-27 15:53:50", "date": "2018-11-27", "time": "15:53:50" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688308", "datetime": "2019-04-11 18:30:12", "date": "2019-04-11", "time": "18:30:12" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688309", "datetime": "2015-06-06 20:01:06", "date": "2015-06-06", "time": "20:01:06" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830a", "datetime": "2019-03-14 22:01:52", "date": "2019-03-14", "time": "22:01:52" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830b", "datetime": "2012-07-13 21:43:57", "date": "2012-07-13", "time": "21:43:57" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830c", "datetime": "2016-12-24 02:27:31", "date": "2016-12-24", "time": "02:27:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830d", "datetime": "2017-08-31 00:35:26", "date": "2017-08-31", "time": "00:35:26" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830e", "datetime": "2013-04-13 12:35:33", "date": "2013-04-13", "time": "12:35:33" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368830f", "datetime": "2013-08-19 23:35:49", "date": "2013-08-19", "time": "23:35:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688310", "datetime": "2013-10-04 19:14:56", "date": "2013-10-04", "time": "19:14:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688311", "datetime": "2016-09-10 19:26:19", "date": "2016-09-10", "time": "19:26:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688312", "datetime": "2018-09-08 14:15:37", "date": "2018-09-08", "time": "14:15:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688313", "datetime": "2019-09-13 22:47:25", "date": "2019-09-13", "time": "22:47:25" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688314", "datetime": "2019-06-05 18:22:49", "date": "2019-06-05", "time": "18:22:49" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688315", "datetime": "2015-03-16 23:51:16", "date": "2015-03-16", "time": "23:51:16" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688316", "datetime": "2015-12-21 04:48:01", "date": "2015-12-21", "time": "04:48:01" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688317", "datetime": "2016-10-28 20:22:42", "date": "2016-10-28", "time": "20:22:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688318", "datetime": "2016-10-28 20:23:00", "date": "2016-10-28", "time": "20:23:00" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688319", "datetime": "2014-08-07 18:30:48", "date": "2014-08-07", "time": "18:30:48" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831a", "datetime": "2014-09-16 20:41:45", "date": "2014-09-16", "time": "20:41:45" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831b", "datetime": "2014-10-12 23:22:27", "date": "2014-10-12", "time": "23:22:27" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831c", "datetime": "2015-07-21 20:43:56", "date": "2015-07-21", "time": "20:43:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831d", "datetime": "2015-07-21 20:45:07", "date": "2015-07-21", "time": "20:45:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831e", "datetime": "2015-05-02 19:49:05", "date": "2015-05-02", "time": "19:49:05" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368831f", "datetime": "2015-05-06 03:52:18", "date": "2015-05-06", "time": "03:52:18" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688320", "datetime": "2015-09-26 01:13:19", "date": "2015-09-26", "time": "01:13:19" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688321", "datetime": "2019-04-04 22:02:37", "date": "2019-04-04", "time": "22:02:37" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688322", "datetime": "2011-12-21 19:02:51", "date": "2011-12-21", "time": "19:02:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688323", "datetime": "2012-04-15 04:21:39", "date": "2012-04-15", "time": "04:21:39" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688324", "datetime": "2012-04-15 14:23:56", "date": "2012-04-15", "time": "14:23:56" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688325", "datetime": "2013-06-30 22:39:51", "date": "2013-06-30", "time": "22:39:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688326", "datetime": "2013-10-04 20:34:13", "date": "2013-10-04", "time": "20:34:13" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688327", "datetime": "2014-07-16 02:28:40", "date": "2014-07-16", "time": "02:28:40" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688328", "datetime": "2018-06-13 20:16:07", "date": "2018-06-13", "time": "20:16:07" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688329", "datetime": "2011-10-08 12:02:23", "date": "2011-10-08", "time": "12:02:23" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832a", "datetime": "2014-08-18 02:11:11", "date": "2014-08-18", "time": "02:11:11" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832b", "datetime": "2016-01-07 05:27:51", "date": "2016-01-07", "time": "05:27:51" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832c", "datetime": "2016-10-21 20:15:55", "date": "2016-10-21", "time": "20:15:55" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832d", "datetime": "2016-12-01 03:57:10", "date": "2016-12-01", "time": "03:57:10" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832e", "datetime": "2016-12-29 01:54:42", "date": "2016-12-29", "time": "01:54:42" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c368832f", "datetime": "2018-07-22 19:55:31", "date": "2018-07-22", "time": "19:55:31" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688330", "datetime": "2018-09-07 01:42:54", "date": "2018-09-07", "time": "01:42:54" },
+ { "marker": "4fa1e250-7a68-3adb-04ec-6569c3688331", "datetime": "2019-03-08 03:41:06", "date": "2019-03-08", "time": "03:41:06" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848381", "datetime": "2019-06-07 17:54:58", "date": "2019-06-07", "time": "17:54:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848382", "datetime": "2015-04-02 21:45:17", "date": "2015-04-02", "time": "21:45:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848383", "datetime": "2014-01-28 20:56:04", "date": "2014-01-28", "time": "20:56:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848384", "datetime": "2014-11-16 16:11:58", "date": "2014-11-16", "time": "16:11:58" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848385", "datetime": "2015-11-15 19:21:53", "date": "2015-11-15", "time": "19:21:53" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848386", "datetime": "2015-11-15 19:33:39", "date": "2015-11-15", "time": "19:33:39" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848387", "datetime": "2018-09-29 18:55:17", "date": "2018-09-29", "time": "18:55:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848388", "datetime": "2018-10-20 16:48:05", "date": "2018-10-20", "time": "16:48:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848389", "datetime": "2018-10-20 22:20:24", "date": "2018-10-20", "time": "22:20:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838a", "datetime": "2011-04-23 21:11:22", "date": "2011-04-23", "time": "21:11:22" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838b", "datetime": "2014-05-04 19:42:48", "date": "2014-05-04", "time": "19:42:48" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838c", "datetime": "2014-05-11 19:16:08", "date": "2014-05-11", "time": "19:16:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838d", "datetime": "2014-06-04 19:14:18", "date": "2014-06-04", "time": "19:14:18" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838e", "datetime": "2015-12-05 19:22:42", "date": "2015-12-05", "time": "19:22:42" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784838f", "datetime": "2017-05-15 23:19:00", "date": "2017-05-15", "time": "23:19:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848390", "datetime": "2017-05-19 14:30:16", "date": "2017-05-19", "time": "14:30:16" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848391", "datetime": "2017-05-19 14:30:25", "date": "2017-05-19", "time": "14:30:25" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848392", "datetime": "2017-08-28 15:49:37", "date": "2017-08-28", "time": "15:49:37" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848393", "datetime": "2017-09-20 20:19:51", "date": "2017-09-20", "time": "20:19:51" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848394", "datetime": "2017-10-01 16:31:05", "date": "2017-10-01", "time": "16:31:05" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848395", "datetime": "2017-10-01 16:56:27", "date": "2017-10-01", "time": "16:56:27" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848396", "datetime": "2017-12-27 23:33:20", "date": "2017-12-27", "time": "23:33:20" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848397", "datetime": "2011-09-24 21:37:32", "date": "2011-09-24", "time": "21:37:32" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848398", "datetime": "2014-03-10 20:20:07", "date": "2014-03-10", "time": "20:20:07" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f067848399", "datetime": "2015-05-27 00:40:24", "date": "2015-05-27", "time": "00:40:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839a", "datetime": "2015-08-29 17:58:15", "date": "2015-08-29", "time": "17:58:15" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839b", "datetime": "2018-03-16 15:03:26", "date": "2018-03-16", "time": "15:03:26" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839c", "datetime": "2013-10-22 16:49:21", "date": "2013-10-22", "time": "16:49:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839d", "datetime": "2014-11-21 17:39:24", "date": "2014-11-21", "time": "17:39:24" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839e", "datetime": "2015-04-11 13:14:14", "date": "2015-04-11", "time": "13:14:14" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f06784839f", "datetime": "2015-11-21 16:05:56", "date": "2015-11-21", "time": "16:05:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a0", "datetime": "2016-05-06 14:10:04", "date": "2016-05-06", "time": "14:10:04" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a1", "datetime": "2017-08-09 15:15:10", "date": "2017-08-09", "time": "15:15:10" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a2", "datetime": "2017-10-21 15:12:56", "date": "2017-10-21", "time": "15:12:56" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a3", "datetime": "2015-12-03 18:44:00", "date": "2015-12-03", "time": "18:44:00" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a4", "datetime": "2016-03-17 18:19:21", "date": "2016-03-17", "time": "18:19:21" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a5", "datetime": "2016-11-02 15:58:38", "date": "2016-11-02", "time": "15:58:38" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a6", "datetime": "2019-02-27 14:03:08", "date": "2019-02-27", "time": "14:03:08" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a7", "datetime": "2015-05-29 16:46:17", "date": "2015-05-29", "time": "16:46:17" },
+ { "marker": "991ed2f7-3374-33a5-ee65-77f0678483a8", "datetime": "2015-06-01 15:03:53", "date": "2015-06-01", "time": "15:03:53" },
+ { "marker": "BAD_ENTRY_1", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_2", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_3", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_4", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_5", "datetime": "NA", "date": "NA", "time": "NA" },
+ { "marker": "BAD_ENTRY_6", "datetime": "NA", "date": "NA", "time": "NA" }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
new file mode 100644
index 0000000..0d37224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.4.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp
new file mode 100644
index 0000000..bb9eea4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND C.business_id = "--Ni3oJ4VOqfOEu7Sj2Vzg";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp
new file mode 100644
index 0000000..a66da8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT M.marker
+FROM YelpCheckinDateMarkers M, YelpCheckin C, C.dates D
+WHERE D /*+ indexnl */ = M.datetime
+ AND M.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp
new file mode 100644
index 0000000..8bdf70e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/join-unnest-queries/with-open-index/with-open-index.7.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckinDateMarkers M
+INNER JOIN (
+ SELECT VALUE D
+ FROM YelpCheckin C, C.dates D
+) AS CD ON CD /*+ indexnl */ = M.datetime;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..5e18a83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp
new file mode 100644
index 0000000..8bb71f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-1/use-case-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..fcdcde7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp
new file mode 100644
index 0000000..8bb71f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-2/use-case-2.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..4b98d54
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->record->atomic index, and a composite two-field array index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date);
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp
new file mode 100644
index 0000000..145f107
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDate" AND
+ D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..20d1175
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND
+ D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..205d0bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/use-case-4/use-case-4.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..22c7899
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
new file mode 100644
index 0000000..27bc78a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a composite array->record->atomic index on two array fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ t: { time: string }
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date, t.time);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
new file mode 100644
index 0000000..643dad3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..da4ad89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for both a composite atomic, array->atomic and array->atomic, atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id);
+CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id, UNNEST dates);
+CREATE INDEX IdxYelpCheckinDatesBusiness ON YelpCheckin (UNNEST dates, business_id);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp
new file mode 100644
index 0000000..6a693ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpBusinessCheckinDates" AND
+ D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
new file mode 100644
index 0000000..688d2a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/closed/with-composite-sk/with-composite-sk.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDatesBusiness" AND
+ D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..62d70c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates: string) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-1/use-case-1.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..0edb2cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-2/use-case-2.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..733d2d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for an array->record->atomic index, and a composite two-field array index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string ?) ;
+CREATE INDEX IdxYelpCheckinDate ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
+CREATE INDEX IdxYelpCheckinTime ON YelpCheckin (UNNEST checkin_times SELECT time : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp
new file mode 100644
index 0000000..f705a94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.2.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDate" AND
+ D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp
new file mode 100644
index 0000000..fc849a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-3/use-case-3.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinTime" AND
+ D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..8ea3bfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a array->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinBusinessID ON YelpCheckin (business_id : string?) ;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/use-case-4/use-case-4.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..c9b993c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a record->record->record->array->atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
new file mode 100644
index 0000000..822f970
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDates" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
new file mode 100644
index 0000000..5747a3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for a composite array->record->atomic index on two array fields.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpCheckinDateTime ON YelpCheckin (UNNEST checkin_times SELECT date : string , t.time : string ) ;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
new file mode 100644
index 0000000..643dad3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.2.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpCheckinDateTime" AND D.DataverseName = "TestYelp";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..2a9497a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify the index metadata for both a composite atomic, array->atomic and array->atomic, atomic index.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
+CREATE INDEX IdxYelpBusinessCheckinDates ON YelpCheckin (business_id : string ?, UNNEST dates : string ?) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
new file mode 100644
index 0000000..bee279e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/metadata/open/with-composite-sk/with-composite-sk.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT D.SearchKey, D.SearchKeyElements, D.SearchKeyType
+FROM Metadata.`Index` D
+WHERE D.IndexName = "IdxYelpBusinessCheckinDates" AND D.DataverseName = "TestYelp";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..19b7ff7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ * 1) Implicit existential quantification w/ membership query.
+ * 2) Explicit existential quantification query, performing a range search.
+ * 3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-1/use-case-1.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..fd32cb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ * 1) Implicit existential quantification w/ membership query.
+ * 2) Explicit existential quantification query, performing a range search.
+ * 3) Universal quantification query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..ff3b923
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26" IN C.checkin_times.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..cea9d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
new file mode 100644
index 0000000..c9652f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-2/use-case-2.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.checkin_times.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..8b4c15b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ * 1) Explicit existential quantification query, performing a point search.
+ * 2) Explicit existential quantification query, performing a point search on two fields inside the array.
+ * 3) Universal quantification query, peforming a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..e21053a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..30547411
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES D.date = "2016-04-26" AND D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..6a73591
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY D IN C.checkin_times
+SATISFIES D.date BETWEEN "2016" AND "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..65c16dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ * 1) Nested existential quantification (implicit using membership and explicit outer) query.
+ * 2) Nested existential quantification (explicit inner and outer) query.
+ * 3) Universal quantification containing explicit existential quantification query.
+ * 4) Nested universal quantification query.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..1832378
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME D IN C.checkin_times
+SATISFIES "2016-04-26" IN D.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..ea254b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE SOME CT IN C.checkin_times
+SATISFIES (
+ SOME D IN CT.dates
+ SATISFIES D = "2016-04-26"
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
new file mode 100644
index 0000000..490dbdd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.6.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY CT IN C.checkin_times
+SATISFIES (
+ SOME D IN CT.dates
+ SATISFIES "2019-06-07" = D
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
new file mode 100644
index 0000000..4032e84
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/use-case-4/use-case-4.7.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE EVERY CT IN C.checkin_times
+SATISFIES (
+ EVERY D IN CT.dates
+ SATISFIES D > "2019-06-05"
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..0b3bc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index
+ * on a dataset with a composite PK.
+ * 1) Implicit existential quantification w/ membership query.
+ * 2) Explicit existential quantification query, performing a range search.
+ * 3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..4f8b1f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an open array->atomic index.
+ * 1) Implicit existential quantification w/ membership query.
+ * 2) Explicit existential quantification query, performing a range search.
+ * 3) Universal quantification query, perform a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
new file mode 100644
index 0000000..0d37224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp
new file mode 100644
index 0000000..dc5fc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C
+WHERE "2016-04-26 19:49:16" IN C.dates;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp
new file mode 100644
index 0000000..2921d18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE SOME D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
new file mode 100644
index 0000000..e64a93c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/with-open-index/with-open-index.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C
+WHERE EVERY D IN C.dates
+SATISFIES D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..ae09aa5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..58dc001
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..0ab72fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..9887068
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..d20997e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ * 3) UNNEST query, performing a point search with an additional predicate after the UNNEST.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..ee911b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..912cfcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..4ab2742
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date = "2016-04-26" AND
+ D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..809e207
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ dates: [string],
+ times: [string]
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..05a7c76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..cd5b2c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..2f858dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..5caafd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->record->record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_data: {
+ checkin_temporal: {
+ checkin_times: {
+ dates: [string],
+ times: [string]
+ }
+ }
+ }
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..5ef37bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
new file mode 100644
index 0000000..847e73f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..b375231
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a composite PK.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: smallint,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id, business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
new file mode 100644
index 0000000..6498c14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.update.sqlpp
@@ -0,0 +1,306 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "checkin_id": 1,
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "checkin_id": 2,
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "checkin_id": 3,
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "checkin_id": 4,
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "checkin_id": 5,
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "checkin_id": 6,
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "checkin_id": 7,
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "checkin_id": 8,
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "checkin_id": 9,
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "checkin_id": 10,
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "checkin_id": 11,
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "checkin_id": 12,
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "checkin_id": 13,
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "checkin_id": 14,
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "checkin_id": 15,
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "checkin_id": 16,
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "checkin_id": 17,
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "checkin_id": 18,
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "checkin_id": 19,
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "checkin_id": 20,
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "checkin_id": 21,
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "checkin_id": 22,
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "checkin_id": 23,
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "checkin_id": 24,
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "checkin_id": 25,
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "checkin_id": 26,
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "checkin_id": 27,
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "checkin_id": 28,
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "checkin_id": 29,
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "checkin_id": 30,
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..fc115e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index on both array fields + an atomic field.
+ * 1) UNNEST query, performing a composite point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ checkin_times: [{
+ date: string,
+ time: string
+ }]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
new file mode 100644
index 0000000..20de338
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date, time);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp
new file mode 100644
index 0000000..3023de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ "19:49:16" = D.time AND
+ "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..2d7d649
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index with a dataset filter field.
+ * 1) UNNEST query, performing a point search with the inclusion of the filter field.
+ * 2) UNNEST query, performing a range search without the inclusion of the filter field.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid,
+ business_id: string,
+ dates: [string]
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED WITH FILTER ON business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
new file mode 100644
index 0000000..a0fc1bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
new file mode 100644
index 0000000..bd0f395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D AND
+ C.business_id = "--1UhMGODdWsrMastO9DZw";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp
new file mode 100644
index 0000000..c87f4ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using an array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
new file mode 100644
index 0000000..b75bf1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "dates": [
+ "2016-04-26 19:49:16",
+ "2016-08-30 18:36:57",
+ "2016-10-15 02:45:18",
+ "2016-11-18 01:54:50",
+ "2017-04-20 18:39:06",
+ "2017-05-03 17:58:02",
+ "2019-03-19 22:04:48"
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "dates": [
+ "2018-05-25 19:52:07",
+ "2018-09-18 16:09:44",
+ "2019-10-18 21:29:09"
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "dates": [
+ "2019-06-07 17:54:58"
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "dates": [
+ "2011-05-03 20:54:05",
+ "2011-08-23 20:49:45",
+ "2014-12-04 06:13:01",
+ "2016-11-16 19:25:55"
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "dates": [
+ "2016-06-18 21:35:45",
+ "2016-10-15 18:17:51"
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "dates": [
+ "2015-04-02 21:45:17"
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "dates": [
+ "2014-08-27 17:49:18",
+ "2015-12-19 21:30:31",
+ "2018-11-27 15:53:50"
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "dates": [
+ "2014-01-28 20:56:04",
+ "2014-11-16 16:11:58",
+ "2015-11-15 19:21:53",
+ "2015-11-15 19:33:39"
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "dates": [
+ "2019-04-11 18:30:12"
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "dates": [
+ "2015-06-06 20:01:06",
+ "2019-03-14 22:01:52"
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "dates": [
+ "2018-09-29 18:55:17",
+ "2018-10-20 16:48:05",
+ "2018-10-20 22:20:24"
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "dates": [
+ "2011-04-23 21:11:22",
+ "2014-05-04 19:42:48",
+ "2014-05-11 19:16:08",
+ "2014-06-04 19:14:18",
+ "2015-12-05 19:22:42",
+ "2017-05-15 23:19:00"
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "dates": [
+ "2012-07-13 21:43:57",
+ "2016-12-24 02:27:31",
+ "2017-08-31 00:35:26"
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "dates": [
+ "2013-04-13 12:35:33",
+ "2013-08-19 23:35:49",
+ "2013-10-04 19:14:56"
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "dates": [
+ "2016-09-10 19:26:19",
+ "2018-09-08 14:15:37",
+ "2019-09-13 22:47:25"
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "dates": [
+ "2017-05-19 14:30:16",
+ "2017-05-19 14:30:25",
+ "2017-08-28 15:49:37",
+ "2017-09-20 20:19:51",
+ "2017-10-01 16:31:05",
+ "2017-10-01 16:56:27",
+ "2017-12-27 23:33:20"
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "dates": [
+ "2019-06-05 18:22:49"
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "dates": [
+ "2011-09-24 21:37:32",
+ "2014-03-10 20:20:07",
+ "2015-05-27 00:40:24",
+ "2015-08-29 17:58:15",
+ "2018-03-16 15:03:26"
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "dates": [
+ "2015-03-16 23:51:16",
+ "2015-12-21 04:48:01",
+ "2016-10-28 20:22:42",
+ "2016-10-28 20:23:00"
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "dates": [
+ "2013-10-22 16:49:21",
+ "2014-11-21 17:39:24"
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "dates": [
+ "2014-08-07 18:30:48",
+ "2014-09-16 20:41:45",
+ "2014-10-12 23:22:27",
+ "2015-07-21 20:43:56",
+ "2015-07-21 20:45:07"
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "dates": [
+ "2015-05-02 19:49:05",
+ "2015-05-06 03:52:18",
+ "2015-09-26 01:13:19"
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "dates": [
+ "2015-04-11 13:14:14",
+ "2015-11-21 16:05:56",
+ "2016-05-06 14:10:04",
+ "2017-08-09 15:15:10",
+ "2017-10-21 15:12:56"
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "dates": [
+ "2015-12-03 18:44:00",
+ "2016-03-17 18:19:21",
+ "2016-11-02 15:58:38"
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "dates": [
+ "2019-04-04 22:02:37"
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "dates": [
+ "2019-02-27 14:03:08"
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "dates": [
+ "2011-12-21 19:02:51",
+ "2012-04-15 04:21:39",
+ "2012-04-15 14:23:56",
+ "2013-06-30 22:39:51",
+ "2013-10-04 20:34:13",
+ "2014-07-16 02:28:40"
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "dates": [
+ "2018-06-13 20:16:07"
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "dates": [
+ "2015-05-29 16:46:17",
+ "2015-06-01 15:03:53"
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "dates": [
+ "2011-10-08 12:02:23",
+ "2014-08-18 02:11:11",
+ "2016-01-07 05:27:51",
+ "2016-10-21 20:15:55",
+ "2016-12-01 03:57:10",
+ "2016-12-29 01:54:42",
+ "2018-07-22 19:55:31",
+ "2018-09-07 01:42:54",
+ "2019-03-08 03:41:06"
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
new file mode 100644
index 0000000..0d37224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp
new file mode 100644
index 0000000..ec79bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.dates D
+WHERE "2016-04-26 19:49:16" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp
new file mode 100644
index 0000000..cba5055
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-1/use-case-1.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp
new file mode 100644
index 0000000..6dbc88c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
new file mode 100644
index 0000000..2549e8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.update.sqlpp
@@ -0,0 +1,497 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
new file mode 100644
index 0000000..abacfe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times.dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp
new file mode 100644
index 0000000..0f23cd4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp
new file mode 100644
index 0000000..9887068
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-2/use-case-2.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp
new file mode 100644
index 0000000..ca44009
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ * 3) UNNEST query, performing a point search with an additional predicate after the UNNEST.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
new file mode 100644
index 0000000..9594cf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times SELECT date : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp
new file mode 100644
index 0000000..912cfcb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp
new file mode 100644
index 0000000..4ab2742
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-3/use-case-3.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date = "2016-04-26" AND
+ D.time = "19:49:16";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp
new file mode 100644
index 0000000..f1cc5fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
new file mode 100644
index 0000000..7fcf9e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.update.sqlpp
@@ -0,0 +1,425 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {
+ "dates": ["2016-04-26", "2016-08-30", "2016-10-15", "2016-11-18"],
+ "times": ["19:49:16", "18:36:57", "02:45:18", "01:54:50"]
+ },
+ {
+ "dates": ["2017-04-20", "2017-05-03"],
+ "times": ["18:39:06", "17:58:02"]
+ },
+ {
+ "dates": ["2019-03-19"],
+ "times": ["22:04:48"]
+ }
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {
+ "dates": ["2018-05-25", "2018-09-18"],
+ "times": ["19:52:07", "16:09:44"]
+ },
+ {
+ "dates": ["2019-10-18"],
+ "times": ["21:29:09"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-07"],
+ "times": ["17:54:58"]
+ }
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {
+ "dates": ["2011-05-03", "2011-08-23"],
+ "times": ["20:54:05", "20:49:45"]
+ },
+ {
+ "dates": ["2014-12-04"],
+ "times": ["06:13:01"]
+ },
+ {
+ "dates": ["2016-11-16"],
+ "times": ["19:25:55"]
+ }
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {
+ "dates": ["2016-06-18", "2016-10-15"],
+ "times": ["21:35:45", "18:17:51"]
+ }
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-02"],
+ "times": ["21:45:17"]
+ }
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-27"],
+ "times": ["17:49:18"]
+ },
+ {
+ "dates": ["2015-12-19"],
+ "times": ["21:30:31"]
+ },
+ {
+ "dates": ["2018-11-27"],
+ "times": ["15:53:50"]
+ }
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {
+ "dates": ["2014-01-28", "2014-11-16"],
+ "times": ["20:56:04", "16:11:58"]
+ },
+ {
+ "dates": ["2015-11-15", "2015-11-15"],
+ "times": ["19:21:53", "19:33:39"]
+ }
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-11"],
+ "times": ["18:30:12"]
+ }
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-06-06"],
+ "times": ["20:01:06"]
+ },
+ {
+ "dates": ["2019-03-14"],
+ "times": ["22:01:52"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {
+ "dates": ["2018-09-29", "2018-10-20", "2018-10-20"],
+ "times": ["18:55:17", "16:48:05", "22:20:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {
+ "dates": ["2011-04-23"],
+ "times": ["21:11:22"]
+ },
+ {
+ "dates": ["2014-05-04", "2014-05-11", "2014-06-04"],
+ "times": ["19:42:48", "19:16:08", "19:14:18"]
+ },
+ {
+ "dates": ["2015-12-05"],
+ "times": ["19:22:42"]
+ },
+ {
+ "dates": ["2017-05-15"],
+ "times": ["23:19:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {
+ "dates": ["2012-07-13"],
+ "times": ["21:43:57"]
+ },
+ {
+ "dates": ["2016-12-24"],
+ "times": ["02:27:31"]
+ },
+ {
+ "dates": ["2017-08-31"],
+ "times": ["00:35:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-04-13", "2013-08-19", "2013-10-04"],
+ "times": ["12:35:33", "23:35:49", "19:14:56"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {
+ "dates": ["2016-09-10"],
+ "times": ["19:26:19"]
+ },
+ {
+ "dates": ["2018-09-08"],
+ "times": ["14:15:37"]
+ },
+ {
+ "dates": ["2019-09-13"],
+ "times": ["22:47:25"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {
+ "dates": ["2017-05-19", "2017-05-19", "2017-08-28", "2017-09-20", "2017-10-01", "2017-10-01", "2017-12-27"],
+ "times": ["14:30:16", "14:30:25", "15:49:37", "20:19:51", "16:31:05", "16:56:27", "23:33:20"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {
+ "dates": ["2019-06-05"],
+ "times": ["18:22:49"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {
+ "dates": ["2011-09-24"],
+ "times": ["21:37:32"]
+ },
+ {
+ "dates": ["2014-03-10"],
+ "times": ["20:20:07"]
+ },
+ {
+ "dates": ["2015-05-27", "2015-08-29"],
+ "times": ["00:40:24", "17:58:15"]
+ },
+ {
+ "dates": ["2018-03-16"],
+ "times": ["15:03:26"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {
+ "dates": ["2015-03-16", "2015-12-21"],
+ "times": ["23:51:16", "04:48:01"]
+ },
+ {
+ "dates": ["2016-10-28", "2016-10-28"],
+ "times": ["20:22:42", "20:23:00"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {
+ "dates": ["2013-10-22"],
+ "times": ["16:49:21"]
+ },
+ {
+ "dates": ["2014-11-21"],
+ "times": ["17:39:24"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {
+ "dates": ["2014-08-07", "2014-09-16", "2014-10-12"],
+ "times": ["18:30:48", "20:41:45", "23:22:27"]
+ },
+ {
+ "dates": ["2015-07-21", "2015-07-21"],
+ "times": ["20:43:56", "20:45:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-02", "2015-05-06", "2015-09-26"],
+ "times": ["19:49:05", "03:52:18", "01:13:19"]
+ }
+
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-04-11", "2015-11-21"],
+ "times": ["13:14:14", "16:05:56"]
+ },
+ {
+ "dates": ["2016-05-06"],
+ "times": ["14:10:04"]
+ },
+ {
+ "dates": ["2017-08-09", "2017-10-21"],
+ "times": ["15:15:10", "15:12:56"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {
+ "dates": ["2015-12-03"],
+ "times": ["18:44:00"]
+ },
+ {
+ "dates": ["2016-03-17", "2016-11-02"],
+ "times": ["18:19:21", "15:58:38"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {
+ "dates": ["2019-04-04"],
+ "times": ["22:02:37"]
+ }
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {
+ "dates": ["2019-02-27"],
+ "times": ["14:03:08"]
+ }
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {
+ "dates": ["2011-12-21"],
+ "times": ["19:02:51"]
+ },
+ {
+ "dates": ["2012-04-15", "2012-04-15"],
+ "times": ["04:21:39", "14:23:56"]
+ },
+ {
+ "dates": ["2013-06-30", "2013-10-04"],
+ "times": ["22:39:51", "20:34:13"]
+ },
+ {
+ "dates": ["2014-07-16"],
+ "times": ["02:28:40"]
+ }
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {
+ "dates": ["2018-06-13"],
+ "times": ["20:16:07"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {
+ "dates": ["2015-05-29", "2015-06-01"],
+ "times": ["16:46:17", "15:03:53"]
+ }
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {
+ "dates": ["2011-10-08"],
+ "times": ["12:02:23"]
+ },
+ {
+ "dates": ["2014-08-18"],
+ "times": ["02:11:11"]
+ },
+ {
+ "dates": ["2016-01-07", "2016-10-21", "2016-12-01", "2016-12-29"],
+ "times": ["05:27:51", "20:15:55", "03:57:10", "01:54:42"]
+ },
+ {
+ "dates": ["2018-07-22", "2018-09-07"],
+ "times": ["19:55:31", "01:42:54"]
+ },
+ {
+ "dates": ["2019-03-08"],
+ "times": ["03:41:06"]
+ }
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
new file mode 100644
index 0000000..c590ecb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_times UNNEST dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp
new file mode 100644
index 0000000..cd5b2c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE "2016-04-26" = D;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp
new file mode 100644
index 0000000..2f858dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/use-case-4/use-case-4.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times CT, CT.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
new file mode 100644
index 0000000..a6e46aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a record->record->record->array->atomic index.
+ * 1) UNNEST query, performing a point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
new file mode 100644
index 0000000..884c224
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.update.sqlpp
@@ -0,0 +1,617 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-04-26",
+ "2016-08-30",
+ "2016-10-15",
+ "2016-11-18",
+ "2017-04-20",
+ "2017-05-03",
+ "2019-03-19"
+ ],
+ "times": [
+ "19:49:16",
+ "18:36:57",
+ "02:45:18",
+ "01:54:50",
+ "18:39:06",
+ "17:58:02",
+ "22:04:48"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-05-25",
+ "2018-09-18",
+ "2019-10-18"
+ ],
+ "times": [
+ "19:52:07",
+ "16:09:44",
+ "21:29:09"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-07"
+ ],
+ "times": [
+ "17:54:58"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-05-03",
+ "2011-08-23",
+ "2014-12-04",
+ "2016-11-16"
+ ],
+ "times": [
+ "20:54:05",
+ "20:49:45",
+ "06:13:01",
+ "19:25:55"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-06-18",
+ "2016-10-15"
+ ],
+ "times": [
+ "21:35:45",
+ "18:17:51"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-02"
+ ],
+ "times": [
+ "21:45:17"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-27",
+ "2015-12-19",
+ "2018-11-27"
+ ],
+ "times": [
+ "17:49:18",
+ "21:30:31",
+ "15:53:50"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-01-28",
+ "2014-11-16",
+ "2015-11-15",
+ "2015-11-15"
+ ],
+ "times": [
+ "20:56:04",
+ "16:11:58",
+ "19:21:53",
+ "19:33:39"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-11"
+ ],
+ "times": [
+ "18:30:12"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-06-06",
+ "2019-03-14"
+ ],
+ "times": [
+ "20:01:06",
+ "22:01:52"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-09-29",
+ "2018-10-20",
+ "2018-10-20"
+ ],
+ "times": [
+ "18:55:17",
+ "16:48:05",
+ "22:20:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-04-23",
+ "2014-05-04",
+ "2014-05-11",
+ "2014-06-04",
+ "2015-12-05",
+ "2017-05-15"
+ ],
+ "times": [
+ "21:11:22",
+ "19:42:48",
+ "19:16:08",
+ "19:14:18",
+ "19:22:42",
+ "23:19:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2012-07-13",
+ "2016-12-24",
+ "2017-08-31"
+ ],
+ "times": [
+ "21:43:57",
+ "02:27:31",
+ "00:35:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-04-13",
+ "2013-08-19",
+ "2013-10-04"
+ ],
+ "times": [
+ "12:35:33",
+ "23:35:49",
+ "19:14:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2016-09-10",
+ "2018-09-08",
+ "2019-09-13"
+ ],
+ "times": [
+ "19:26:19",
+ "14:15:37",
+ "22:47:25"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2017-05-19",
+ "2017-05-19",
+ "2017-08-28",
+ "2017-09-20",
+ "2017-10-01",
+ "2017-10-01",
+ "2017-12-27"
+ ],
+ "times": [
+ "14:30:16",
+ "14:30:25",
+ "15:49:37",
+ "20:19:51",
+ "16:31:05",
+ "16:56:27",
+ "23:33:20"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-06-05"
+ ],
+ "times": [
+ "18:22:49"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-09-24",
+ "2014-03-10",
+ "2015-05-27",
+ "2015-08-29",
+ "2018-03-16"
+ ],
+ "times": [
+ "21:37:32",
+ "20:20:07",
+ "00:40:24",
+ "17:58:15",
+ "15:03:26"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-03-16",
+ "2015-12-21",
+ "2016-10-28",
+ "2016-10-28"
+ ],
+ "times": [
+ "23:51:16",
+ "04:48:01",
+ "20:22:42",
+ "20:23:00"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2013-10-22",
+ "2014-11-21"
+ ],
+ "times": [
+ "16:49:21",
+ "17:39:24"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2014-08-07",
+ "2014-09-16",
+ "2014-10-12",
+ "2015-07-21",
+ "2015-07-21"
+ ],
+ "times": [
+ "18:30:48",
+ "20:41:45",
+ "23:22:27",
+ "20:43:56",
+ "20:45:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-02",
+ "2015-05-06",
+ "2015-09-26"
+ ],
+ "times": [
+ "19:49:05",
+ "03:52:18",
+ "01:13:19"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-04-11",
+ "2015-11-21",
+ "2016-05-06",
+ "2017-08-09",
+ "2017-10-21"
+ ],
+ "times": [
+ "13:14:14",
+ "16:05:56",
+ "14:10:04",
+ "15:15:10",
+ "15:12:56"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-12-03",
+ "2016-03-17",
+ "2016-11-02"
+ ],
+ "times": [
+ "18:44:00",
+ "18:19:21",
+ "15:58:38"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-04-04"
+ ],
+ "times": [
+ "22:02:37"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2019-02-27"
+ ],
+ "times": [
+ "14:03:08"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-12-21",
+ "2012-04-15",
+ "2012-04-15",
+ "2013-06-30",
+ "2013-10-04",
+ "2014-07-16"
+ ],
+ "times": [
+ "19:02:51",
+ "04:21:39",
+ "14:23:56",
+ "22:39:51",
+ "20:34:13",
+ "02:28:40"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2018-06-13"
+ ],
+ "times": [
+ "20:16:07"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2015-05-29",
+ "2015-06-01"
+ ],
+ "times": [
+ "16:46:17",
+ "15:03:53"
+ ]
+ }
+ }
+ }
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_data": {
+ "checkin_temporal": {
+ "checkin_times": {
+ "dates": [
+ "2011-10-08",
+ "2014-08-18",
+ "2016-01-07",
+ "2016-10-21",
+ "2016-12-01",
+ "2016-12-29",
+ "2018-07-22",
+ "2018-09-07",
+ "2019-03-08"
+ ],
+ "times": [
+ "12:02:23",
+ "02:11:11",
+ "05:27:51",
+ "20:15:55",
+ "03:57:10",
+ "01:54:42",
+ "19:55:31",
+ "01:42:54",
+ "03:41:06"
+ ]
+ }
+ }
+ }
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
new file mode 100644
index 0000000..88a6782
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDates ON YelpCheckin (UNNEST checkin_data.checkin_temporal.checkin_times.dates : string ) ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
new file mode 100644
index 0000000..cee1887
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.4.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE "2016-04-26" = D;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
new file mode 100644
index 0000000..847e73f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_data.checkin_temporal.checkin_times.dates D
+WHERE D > "2016" AND D < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
new file mode 100644
index 0000000..f3cae85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/*
+ * Description: Verify that we are able to correctly perform the following types of queries using a array->record->atomic index on both array fields + an atomic field.
+ * 1) UNNEST query, performing a composite point search.
+ * 2) UNNEST query, performing a range search.
+ */
+
+DROP DATAVERSE TestYelp IF EXISTS;
+CREATE DATAVERSE TestYelp;
+USE TestYelp;
+
+CREATE TYPE CheckinType AS {
+ checkin_id: uuid
+};
+
+CREATE DATASET YelpCheckin(CheckinType) PRIMARY KEY checkin_id AUTOGENERATED;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
new file mode 100644
index 0000000..02ddf21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.update.sqlpp
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+INSERT INTO YelpCheckin (
+ [
+ {
+ "business_id": "--1UhMGODdWsrMastO9DZw",
+ "checkin_times": [
+ {"date": "2016-04-26", "time": "19:49:16"},
+ {"date": "2016-08-30", "time": "18:36:57"},
+ {"date": "2016-10-15", "time": "02:45:18"},
+ {"date": "2016-11-18", "time": "01:54:50"},
+ {"date": "2017-04-20", "time": "18:39:06"},
+ {"date": "2017-05-03", "time": "17:58:02"},
+ {"date": "2019-03-19", "time": "22:04:48"}
+ ]
+ },
+ {
+ "business_id": "--EF5N7P70J_UYBTPypYlA",
+ "checkin_times": [
+ {"date": "2018-05-25", "time": "19:52:07"},
+ {"date": "2018-09-18", "time": "16:09:44"},
+ {"date": "2019-10-18", "time": "21:29:09"}
+ ]
+ },
+ {
+ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg",
+ "checkin_times": [
+ {"date": "2019-06-07", "time": "17:54:58"}
+ ]
+ },
+ {
+ "business_id": "--Y1Adl1YUWfYIRSd8vkmA",
+ "checkin_times": [
+ {"date": "2011-05-03", "time": "20:54:05"},
+ {"date": "2011-08-23", "time": "20:49:45"},
+ {"date": "2014-12-04", "time": "06:13:01"},
+ {"date": "2016-11-16", "time": "19:25:55"}
+ ]
+ },
+ {
+ "business_id": "--YPwqIlRJrhHkJcjY3eiA",
+ "checkin_times": [
+ {"date": "2016-06-18", "time": "21:35:45"},
+ {"date": "2016-10-15", "time": "18:17:51"}
+ ]
+ },
+ {
+ "business_id": "--e8PjCNhEz32pprnPhCwQ",
+ "checkin_times": [
+ {"date": "2015-04-02", "time": "21:45:17"}
+ ]
+ },
+ {
+ "business_id": "--kinfHwmtdjz03g8B8z8Q",
+ "checkin_times": [
+ {"date": "2014-08-27", "time": "17:49:18"},
+ {"date": "2015-12-19", "time": "21:30:31"},
+ {"date": "2018-11-27", "time": "15:53:50"}
+ ]
+ },
+ {
+ "business_id": "--q6datkI-f0EoVheXNEeQ",
+ "checkin_times": [
+ {"date": "2014-01-28", "time": "20:56:04"},
+ {"date": "2014-11-16", "time": "16:11:58"},
+ {"date": "2015-11-15", "time": "19:21:53"},
+ {"date": "2015-11-15", "time": "19:33:39"}
+ ]
+ },
+ {
+ "business_id": "--qvQS4MigHPykD2GV0-zw",
+ "checkin_times": [
+ {"date": "2019-04-11", "time": "18:30:12"}
+ ]
+ },
+ {
+ "business_id": "--wIGbLEhlpl_UeAIyDmZQ",
+ "checkin_times": [
+ {"date": "2015-06-06", "time": "20:01:06"},
+ {"date": "2019-03-14", "time": "22:01:52"}
+ ]
+ },
+ {
+ "business_id": "-0FA-Qdi3SPYIoJz9UQw-A",
+ "checkin_times": [
+ {"date": "2018-09-29", "time": "18:55:17"},
+ {"date": "2018-10-20", "time": "16:48:05"},
+ {"date": "2018-10-20", "time": "22:20:24"}
+ ]
+ },
+ {
+ "business_id": "-0Hj1hb_XW6ybWq2M7QhGA",
+ "checkin_times": [
+ {"date": "2011-04-23", "time": "21:11:22"},
+ {"date": "2014-05-04", "time": "19:42:48"},
+ {"date": "2014-05-11", "time": "19:16:08"},
+ {"date": "2014-06-04", "time": "19:14:18"},
+ {"date": "2015-12-05", "time": "19:22:42"},
+ {"date": "2017-05-15", "time": "23:19:00"}
+ ]
+ },
+ {
+ "business_id": "-0KMvRFwDWdVBeTpT11iHw",
+ "checkin_times": [
+ {"date": "2012-07-13", "time": "21:43:57"},
+ {"date": "2016-12-24", "time": "02:27:31"},
+ {"date": "2017-08-31", "time": "00:35:26"}
+ ]
+ },
+ {
+ "business_id": "-0LPtgJC31FWMrMv317p0Q",
+ "checkin_times": [
+ {"date": "2013-04-13", "time": "12:35:33"},
+ {"date": "2013-08-19", "time": "23:35:49"},
+ {"date": "2013-10-04", "time": "19:14:56"}
+ ]
+ },
+ {
+ "business_id": "-0M3o2uWBnQZwd3hmfEwuw",
+ "checkin_times": [
+ {"date": "2016-09-10", "time": "19:26:19"},
+ {"date": "2018-09-08", "time": "14:15:37"},
+ {"date": "2019-09-13", "time": "22:47:25"}
+ ]
+ },
+ {
+ "business_id": "-0RRiWDtfnS16AKCtfvBZg",
+ "checkin_times": [
+ {"date": "2017-05-19", "time": "14:30:16"},
+ {"date": "2017-05-19", "time": "14:30:25"},
+ {"date": "2017-08-28", "time": "15:49:37"},
+ {"date": "2017-09-20", "time": "20:19:51"},
+ {"date": "2017-10-01", "time": "16:31:05"},
+ {"date": "2017-10-01", "time": "16:56:27"},
+ {"date": "2017-12-27", "time": "23:33:20"}
+ ]
+ },
+ {
+ "business_id": "-0Soj75v-XoRcf2ERr8Bmg",
+ "checkin_times": [
+ {"date": "2019-06-05", "time": "18:22:49"}
+ ]
+ },
+ {
+ "business_id": "-0ZumLlFjMh4ZW1z2nXGug",
+ "checkin_times": [
+ {"date": "2011-09-24", "time": "21:37:32"},
+ {"date": "2014-03-10", "time": "20:20:07"},
+ {"date": "2015-05-27", "time": "00:40:24"},
+ {"date": "2015-08-29", "time": "17:58:15"},
+ {"date": "2018-03-16", "time": "15:03:26"}
+ ]
+ },
+ {
+ "business_id": "-0aOudcaAyac0VJbMX-L1g",
+ "checkin_times": [
+ {"date": "2015-03-16", "time": "23:51:16"},
+ {"date": "2015-12-21", "time": "04:48:01"},
+ {"date": "2016-10-28", "time": "20:22:42"},
+ {"date": "2016-10-28", "time": "20:23:00"}
+ ]
+ },
+ {
+ "business_id": "-0b86isaXMY0v4g-V8GZ9Q",
+ "checkin_times": [
+ {"date": "2013-10-22", "time": "16:49:21"},
+ {"date": "2014-11-21", "time": "17:39:24"}
+ ]
+ },
+ {
+ "business_id": "-0d-BfFSU0bwLcnMaGRxYw",
+ "checkin_times": [
+ {"date": "2014-08-07", "time": "18:30:48"},
+ {"date": "2014-09-16", "time": "20:41:45"},
+ {"date": "2014-10-12", "time": "23:22:27"},
+ {"date": "2015-07-21", "time": "20:43:56"},
+ {"date": "2015-07-21", "time": "20:45:07"}
+ ]
+ },
+ {
+ "business_id": "-0jz6c3C6i7RG7Ag22K-Pg",
+ "checkin_times": [
+ {"date": "2015-05-02", "time": "19:49:05"},
+ {"date": "2015-05-06", "time": "03:52:18"},
+ {"date": "2015-09-26", "time": "01:13:19"}
+ ]
+ },
+ {
+ "business_id": "-0y3MZU2oYP8r1ruDP1bfQ",
+ "checkin_times": [
+ {"date": "2015-04-11", "time": "13:14:14"},
+ {"date": "2015-11-21", "time": "16:05:56"},
+ {"date": "2016-05-06", "time": "14:10:04"},
+ {"date": "2017-08-09", "time": "15:15:10"},
+ {"date": "2017-10-21", "time": "15:12:56"}
+ ]
+ },
+ {
+ "business_id": "-1BPe8UjF2_l3nVk-DFUjA",
+ "checkin_times": [
+ {"date": "2015-12-03", "time": "18:44:00"},
+ {"date": "2016-03-17", "time": "18:19:21"},
+ {"date": "2016-11-02", "time": "15:58:38"}
+ ]
+ },
+ {
+ "business_id": "-1E2CQu_38mkghvmZgCCRw",
+ "checkin_times": [
+ {"date": "2019-04-04", "time": "22:02:37"}
+ ]
+ },
+ {
+ "business_id": "-1wzk43IZ5D9Ysu6kzb5xA",
+ "checkin_times": [
+ {"date": "2019-02-27", "time": "14:03:08"}
+ ]
+ },
+ {
+ "business_id": "-23R9P2eG7VTc6DVLjFKzA",
+ "checkin_times": [
+ {"date": "2011-12-21", "time": "19:02:51"},
+ {"date": "2012-04-15", "time": "04:21:39"},
+ {"date": "2012-04-15", "time": "14:23:56"},
+ {"date": "2013-06-30", "time": "22:39:51"},
+ {"date": "2013-10-04", "time": "20:34:13"},
+ {"date": "2014-07-16", "time": "02:28:40"}
+ ]
+ },
+ {
+ "business_id": "-26MGfikhJiTfCI-GqmzhQ",
+ "checkin_times": [
+ {"date": "2018-06-13", "time": "20:16:07"}
+ ]
+ },
+ {
+ "business_id": "-2bLuJsMZ0WhI9daurVQNQ",
+ "checkin_times": [
+ {"date": "2015-05-29", "time": "16:46:17"},
+ {"date": "2015-06-01", "time": "15:03:53"}
+ ]
+ },
+ {
+ "business_id": "-2hDBMaza_ldqnZdiU06LQ",
+ "checkin_times": [
+ {"date": "2011-10-08", "time": "12:02:23"},
+ {"date": "2014-08-18", "time": "02:11:11"},
+ {"date": "2016-01-07", "time": "05:27:51"},
+ {"date": "2016-10-21", "time": "20:15:55"},
+ {"date": "2016-12-01", "time": "03:57:10"},
+ {"date": "2016-12-29", "time": "01:54:42"},
+ {"date": "2018-07-22", "time": "19:55:31"},
+ {"date": "2018-09-07", "time": "01:42:54"},
+ {"date": "2019-03-08", "time": "03:41:06"}
+ ]
+ }
+ ]
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
new file mode 100644
index 0000000..a8cf7d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE TestYelp;
+
+CREATE INDEX IdxYelpCheckinDatesTimesBusiness ON YelpCheckin (UNNEST checkin_times SELECT date : string , time : string );
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp
new file mode 100644
index 0000000..3023de1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.4.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT C.business_id
+FROM YelpCheckin C, C.checkin_times D
+WHERE "2016-04-26" = D.date AND
+ "19:49:16" = D.time AND
+ "--1UhMGODdWsrMastO9DZw" = C.business_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp
new file mode 100644
index 0000000..244ae62
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+SET `compiler.arrayindex` "true";
+
+USE TestYelp;
+
+SELECT COUNT(*)
+FROM YelpCheckin C, C.checkin_times D
+WHERE D.date > "2016" AND D.date < "2017";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/delete-syntax-change.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/delete-syntax-change.sqlpp
index 34c2b6f..dc29bbd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/delete-syntax-change.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/delete-syntax-change.sqlpp
@@ -56,7 +56,6 @@
delete from LineItem
where l_orderkey >= 10;
-write output to asterix_nc1:`rttest/dml_delete-syntax-change.adm`
select element c
from LineItem as c
order by c.l_orderkey,c.l_linenumber
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-syntax.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-syntax.sqlpp
index 59edd0e..c8b3330 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-syntax.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-syntax.sqlpp
@@ -45,7 +45,6 @@
select element {'id':3,'name':'Person Three','hobbies':{{'Blues'}}};
insert into testds
select element {'id':4,'name':'Person Four','hobbies':{{'Metal','Jazz'}}};
-write output to asterix_nc1:`rttest/dml_insert-syntax.adm`
select element d
from testds as d
order by d.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp
new file mode 100644
index 0000000..ad34b86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 1: Create dataset and insert data
+
+DROP DATAVERSE MyDataVerse IF EXISTS;
+CREATE DATAVERSE MyDataVerse;
+
+USE MyDataVerse;
+
+DROP DATASET MyMessageDataset IF EXISTS;
+CREATE TYPE MyMessageType AS {
+ myMessageId: int,
+ myMessageBody: string
+};
+CREATE DATASET MyMessageDataset(MyMessageType)
+ PRIMARY KEY myMessageId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp
new file mode 100644
index 0000000..1c6752d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 5: Select with ft_config
+
+USE MyDataVerse;
+
+// Without any ftcontains option
+SELECT VALUE myMessage from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"]) ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp
new file mode 100644
index 0000000..17f3024
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 5a: Select with ft_config
+
+USE MyDataVerse;
+
+// With FT Config my_first_stopword_config and `the`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"], {"mode":"all", "config": "my_first_stopword_config"})
+ ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp
new file mode 100644
index 0000000..6a05ae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 5b: Select with ft_config
+
+USE MyDataVerse;
+
+// With FT Config my_first_stopword_config and `xxx`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["xxx", "cat"], {"mode":"all", "config": "my_first_stopword_config"})
+ ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp
new file mode 100644
index 0000000..c05bd04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 5c: Select with ft_config
+
+USE MyDataVerse;
+
+// With FT Config my_second_stopword_config and `yyy`
+SELECT VALUE myMessage from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["yyy", "cat"], {"mode":"all", "config": "my_second_stopword_config"})
+ ORDER BY myMessage.myMessageId;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp
new file mode 100644
index 0000000..2c10e35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.14.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/////////////////////////////////////////
+// Step 6: Do a full-scan when no index with the same config
+
+USE MyDataVerse;
+
+DROP FULLTEXT CONFIG my_third_stopword_config IF EXISTS;
+CREATE FULLTEXT CONFIG my_third_stopword_config IF NOT EXISTS AS {
+ "tokenizer": "Word",
+ "filterPipeline": []
+}; // No index created with this new config
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp
new file mode 100644
index 0000000..8d5ff33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+/////////////////////////////////////////
+// Step 6: Do a full-scan when no index with the same config
+
+USE MyDataVerse;
+
+SELECT count(myMessage) from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["xxx", "smart", "black"], {"mode":"all", "config": "my_third_stopword_config"});
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp
new file mode 100644
index 0000000..13b4250
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.2.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE MyDataVerse;
+
+INSERT INTO MyMessageDataset ([
+ {
+ "myMessageId": 1,
+ "myMessageBody": "a smart black dog"
+ }
+]);
+
+UPSERT INTO MyMessageDataset ([
+ {
+ "myMessageId": 2,
+ "myMessageBody": "the smart black dog"
+ }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp
new file mode 100644
index 0000000..e475c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.3.ddl.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+USE MyDataVerse;
+
+DROP FULLTEXT CONFIG my_stopword_config IF EXISTS;
+DROP FULLTEXT CONFIG my_first_stopword_config IF EXISTS;
+DROP FULLTEXT CONFIG my_second_stopword_config IF EXISTS;
+DROP FULLTEXT FILTER my_first_stopword_filter IF EXISTS;
+DROP FULLTEXT FILTER my_second_stopword_filter IF EXISTS;
+
+CREATE FULLTEXT FILTER my_first_stopword_filter IF NOT EXISTS AS {
+ "type": "stopwords",
+ "stopwordsList": ["a", "an", "the"]
+};
+CREATE FULLTEXT CONFIG my_first_stopword_config IF NOT EXISTS AS {
+ "tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+ "filterPipeline": ["my_first_stopword_filter"]
+};
+
+CREATE FULLTEXT FILTER my_second_stopword_filter IF NOT EXISTS AS {
+ "type": "stopwords",
+ "stopwordsList": ["xxx", "yyy", "zzz"]
+};
+CREATE FULLTEXT CONFIG my_second_stopword_config IF NOT EXISTS AS {
+ "tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+ "filterPipeline": ["my_second_stopword_filter"]
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp
new file mode 100644
index 0000000..b5c1363
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT VALUE ix FROM Metadata.`FullTextConfig` ix ORDER BY ix.FullTextConfigName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp
new file mode 100644
index 0000000..8e0d694
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT VALUE ix FROM Metadata.`FullTextFilter` ix ORDER BY ix.FullTextFilterName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp
new file mode 100644
index 0000000..fe59571
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.6.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 3: Create 3 FullText Indices
+
+USE MyDataVerse;
+
+DROP INDEX MyMessageDataset.message_ft_index_0 IF EXISTS;
+DROP INDEX MyMessageDataset.message_ft_index_1 IF EXISTS;
+DROP INDEX MyMessageDataset.message_ft_index_2 IF EXISTS;
+
+CREATE INDEX message_ft_index_0 IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT;
+CREATE INDEX message_ft_index_1 IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT USING my_first_stopword_config;
+CREATE INDEX message_ft_index_2 IF NOT EXISTS on MyMessageDataset(myMessageBody) TYPE FULLTEXT USING my_second_stopword_config;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp
new file mode 100644
index 0000000..f770fee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+SELECT Value v from Metadata.`Index` v WHERE v.DataverseName = "MyDataVerse" ORDER BY v. IndexName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp
new file mode 100644
index 0000000..c7489da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.8.update.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 4: Insert after index created
+
+USE MyDataVerse;
+
+UPSERT INTO MyMessageDataset ([
+ {
+ "myMessageId": 3,
+ "myMessageBody": "xxx cute cat"
+ }
+]);
+
+UPSERT INTO MyMessageDataset ([
+ {
+ "myMessageId": 4,
+ "myMessageBody": "yyy crazy cat"
+ }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp
new file mode 100644
index 0000000..0387e37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+///////////////////////////////////////
+// Step 5: Select with ft_config
+
+USE MyDataVerse;
+
+// Without FT Config
+SELECT VALUE myMessage from MyMessageDataset myMessage
+ WHERE ftcontains(myMessage.myMessageBody, ["the", "smart", "black"], {"mode":"all"}) ORDER BY myMessage.myMessageId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/delete-syntax-change.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/delete-syntax-change.sqlpp
index 92446a7..9a08aab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/delete-syntax-change.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/delete-syntax-change.sqlpp
@@ -55,7 +55,6 @@
delete l from LineItem
where (l.l_orderkey >= 10);
-write output to asterix_nc1:`rttest/dml_delete-syntax-change.adm`
select element c
from LineItem as c
order by c.l_orderkey,c.l_linenumber
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/insert-syntax.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/insert-syntax.sqlpp
index 59edd0e..c8b3330 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/insert-syntax.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index-dml/insert-syntax.sqlpp
@@ -45,7 +45,6 @@
select element {'id':3,'name':'Person Three','hobbies':{{'Blues'}}};
insert into testds
select element {'id':4,'name':'Person Four','hobbies':{{'Metal','Jazz'}}};
-write output to asterix_nc1:`rttest/dml_insert-syntax.adm`
select element d
from testds as d
order by d.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_02.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_02.sqlpp
index ddf1991..f0d8a4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_02.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/scan/alltypes_02.sqlpp
@@ -59,7 +59,6 @@
create external dataset `All`(AllType) using localfs((`path`=`asterix_nc1://data/nontagged/allData.json`),(`format`=`adm`));
-write output to asterix_nc1:`rttest/scan_alltypes_02.adm`
select element a
from `All` as a
;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index f4dfdd1..473319c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -8,6 +8,7 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
+ "compiler\.arrayindex" : false,
"compiler\.external\.field\.pushdown" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index 8fc48f9..171ead0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -8,6 +8,7 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
+ "compiler\.arrayindex" : false,
"compiler\.external\.field\.pushdown" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index ed265e5..4a6aff5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -8,6 +8,7 @@
"active\.memory\.global\.budget" : 67108864,
"active\.stop\.timeout" : 3600,
"active\.suspend\.timeout" : 3600,
+ "compiler\.arrayindex" : false,
"compiler\.external\.field\.pushdown" : false,
"compiler\.framesize" : 32768,
"compiler\.groupmemory" : 163840,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/after-index-creation/with-open-index/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..6fa1293
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/bulk-loading/on-index-creation/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-additional-atomic-index/with-additional-atomic-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-composite-sk/with-composite-sk.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/closed/with-filter-fields/with-filter-fields.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm
new file mode 100644
index 0000000..fcc26c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.1.adm
@@ -0,0 +1 @@
+{ "$1": 99 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-additional-atomic-index/with-additional-atomic-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..3e1a847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 7 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/insert-upsert-delete/open/with-composite-sk/with-composite-sk.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-1/with-open-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-3/use-case-3.4.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm
new file mode 100644
index 0000000..965e258
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/use-case-4/use-case-4.4.adm
@@ -0,0 +1 @@
+{ "$1": 119 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm
new file mode 100644
index 0000000..d274256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "marker": "991ed2f7-3374-33a5-ee65-77f067848381" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm
new file mode 100644
index 0000000..5ba90eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "marker": "4fa1e250-7a68-3adb-04ec-6569c36882f5" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm
new file mode 100644
index 0000000..dc7ba8b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/join-unnest-queries/with-open-index/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 101 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..d27a32c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..c4530db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times", "dates" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..0b7a959
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..36048c8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "time" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..40c3a46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ], [ "dates" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..7b80b06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_data", "checkin_temporal", "checkin_times", "dates" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
new file mode 100644
index 0000000..85ce9aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "t", "time" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..18b7416
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ] } ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..5957674
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ] }, [ [ "business_id" ] ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..6e71e97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "dates" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..6eb8776
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times", "dates" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..b33f547
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..194db94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "time" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..d74ae4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ], [ "dates" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..a1aeffe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_data", "checkin_temporal", "checkin_times", "dates" ] ] } ], "SearchKeyType": [ [ "string" ] ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
new file mode 100644
index 0000000..841d0bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-array-different-indicators/with-composite-array-different-indicators.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ { "UnnestList": [ [ "checkin_times" ] ], "ProjectList": [ [ "date" ], [ "t", "time" ] ] } ], "SearchKeyType": [ [ "string", "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..a2eea1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/metadata/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "SearchKey": [ ], "SearchKeyElements": [ [ [ "business_id" ] ], { "UnnestList": [ [ "dates" ] ] } ], "SearchKeyType": [ [ "string" ], [ "string" ] ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-1/use-case-1.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-2/use-case-2.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..d85b4cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--YPwqIlRJrhHkJcjY3eiA" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm
new file mode 100644
index 0000000..e81620a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm
new file mode 100644
index 0000000..e81620a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/use-case-4/use-case-4.4.adm
@@ -0,0 +1 @@
+{ "business_id": "--Ni3oJ4VOqfOEu7Sj2Vzg" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-composite-pk/with-composite-pk.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm
new file mode 100644
index 0000000..eb148f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.2.adm
@@ -0,0 +1 @@
+{ "$1": 9 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm
new file mode 100644
index 0000000..7cc3573
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-quantified-queries/with-open-index/with-open-index.3.adm
@@ -0,0 +1 @@
+{ "$1": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-3-level-record-path/with-3-level-record-path.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-pk/with-composite-pk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/closed/with-filter-fields/with-filter-fields.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-1/use-case-1.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-2/use-case-2.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-3/use-case-3.3.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/use-case-4/use-case-4.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-3-level-record-path/with-3-level-record-path.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm
new file mode 100644
index 0000000..22fc35e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.1.adm
@@ -0,0 +1 @@
+{ "business_id": "--1UhMGODdWsrMastO9DZw" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm
new file mode 100644
index 0000000..9687c35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array-index/select-unnest-queries/open/with-composite-sk/with-composite-sk.2.adm
@@ -0,0 +1 @@
+{ "$1": 18 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm
new file mode 100644
index 0000000..6785ffe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.10.adm
@@ -0,0 +1 @@
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm
new file mode 100644
index 0000000..730d14b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.11.adm
@@ -0,0 +1,2 @@
+{ "myMessageId": 1, "myMessageBody": "a smart black dog" }
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm
new file mode 100644
index 0000000..2c0e461
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.12.adm
@@ -0,0 +1 @@
+{ "myMessageId": 3, "myMessageBody": "xxx cute cat" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm
new file mode 100644
index 0000000..4c485dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.13.adm
@@ -0,0 +1,2 @@
+{ "myMessageId": 3, "myMessageBody": "xxx cute cat" }
+{ "myMessageId": 4, "myMessageBody": "yyy crazy cat" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm
new file mode 100644
index 0000000..3ff59f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.15.adm
@@ -0,0 +1 @@
+{ "$1": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm
new file mode 100644
index 0000000..522e0ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.4.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "MyDataVerse", "FullTextConfigName": "my_first_stopword_config", "Tokenizer": "WORD", "FullTextFilterPipeline": [ "my_first_stopword_filter" ] }
+{ "DataverseName": "MyDataVerse", "FullTextConfigName": "my_second_stopword_config", "Tokenizer": "WORD", "FullTextFilterPipeline": [ "my_second_stopword_filter" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm
new file mode 100644
index 0000000..f117d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.5.adm
@@ -0,0 +1,2 @@
+{ "DataverseName": "MyDataVerse", "FullTextFilterName": "my_first_stopword_filter", "FullTextFilterType": "Stopwords", "StopwordList": [ "a", "an", "the" ] }
+{ "DataverseName": "MyDataVerse", "FullTextFilterName": "my_second_stopword_filter", "FullTextFilterType": "Stopwords", "StopwordList": [ "xxx", "yyy", "zzz" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
new file mode 100644
index 0000000..ba9055b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.7.adm
@@ -0,0 +1,4 @@
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "MyMessageDataset", "IndexStructure": "BTREE", "SearchKey": [ [ "myMessageId" ] ], "IsPrimary": true, "Timestamp": "Sat Nov 14 21:16:57 PST 2020", "PendingOp": 0 }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_0", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0 }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_1", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0, "FullTextConfig": "my_first_stopword_config" }
+{ "DataverseName": "MyDataVerse", "DatasetName": "MyMessageDataset", "IndexName": "message_ft_index_2", "IndexStructure": "SINGLE_PARTITION_WORD_INVIX", "SearchKey": [ [ "myMessageBody" ] ], "IsPrimary": false, "Timestamp": "Sat Nov 14 21:16:59 PST 2020", "PendingOp": 0, "FullTextConfig": "my_second_stopword_config" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm
new file mode 100644
index 0000000..6785ffe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/fulltext/stopwords-full-text-filter-1/stopwords-full-text-filter-1.9.adm
@@ -0,0 +1 @@
+{ "myMessageId": 2, "myMessageBody": "the smart black dog" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
index 1e302da..fca1590 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_azure_blob_storage.xml
@@ -54,7 +54,7 @@
<placeholder name="azureblob-credentialsname-2" value="connectionString" />
<placeholder name="azureblob-credentialsvalue-2" value="%azureblob-connectionstringaccountkey%" />
<output-dir compare="Text">invalid-auth-methods</output-dir>
- <expected-error>ASX1133: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
+ <expected-error>ASX1138: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
@@ -64,13 +64,13 @@
<placeholder name="azureblob-credentialsname-2" value="connectionString" />
<placeholder name="azureblob-credentialsvalue-2" value="%azureblob-connectionstringaccountkey%" />
<output-dir compare="Text">invalid-auth-methods</output-dir>
- <expected-error>ASX1133: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
+ <expected-error>ASX1138: Only a single authentication method is allowed: connectionString, accountName & accountKey, or accountName & sharedAccessSignature</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="external-dataset/azure_blob_storage/auth-methods">
<compilation-unit name="invalid-no-auth">
<output-dir compare="Text">invalid-no-auth</output-dir>
- <expected-error>ASX1134: No authentication parameters provided</expected-error>
+ <expected-error>ASX1139: No authentication parameters provided</expected-error>
</compilation-unit>
</test-case>
</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index c9e37e4..5d1f9fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4397,7 +4397,7 @@
<compilation-unit name="load-with-autogenerated-no-field">
<output-dir compare="Text">load-with-autogenerated-no-field</output-dir>
<expected-error>ASX1014: Field "not_id" is not found</expected-error>
- <expected-error>ASX1014: Field "not_id" is not found (in line 33, at column 1)</expected-error>
+ <expected-error>ASX1014: Field "not_id" is not found</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -5523,6 +5523,11 @@
<output-dir compare="Text">fulltext-index-large-data</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="fulltext">
+ <compilation-unit name="stopwords-full-text-filter-1">
+ <output-dir compare="Text">stopwords-full-text-filter-1</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="global-aggregate">
<test-case FilePath="global-aggregate">
@@ -7027,7 +7032,7 @@
<test-case FilePath="open-index-enforced/error-checking">
<compilation-unit name="index-on-closed-type">
<output-dir compare="Text">index-on-closed-type</output-dir>
- <expected-error>Typed index on "[value]" field could be created only for open datatype</expected-error>
+ <expected-error>ASX1014: Field "value" is not found (in line 33, at column 34)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="open-index-enforced/error-checking">
@@ -7832,6 +7837,416 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="array-index">
+ <test-group name="array-index/error-handling">
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-two-array-fields">
+ <output-dir compare="Text">index-two-array-fields</output-dir>
+ <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+ <expected-error>ASX1079: Compilation error: Cannot create composite index with multiple array fields using different arrays</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="invalid-array-path">
+ <output-dir compare="Text">invalid-array-path</output-dir>
+ <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type CheckinType_checkin_time:</expected-error>
+ <expected-error>ASX0037: Type mismatch: expected value of type array or multiset, but got the value of type string</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-on-closed-array">
+ <output-dir compare="Text">index-on-closed-array</output-dir>
+ <expected-error>ASX1014: Field "date" is not found</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-with-enforced-type">
+ <output-dir compare="Text">index-with-enforced-type</output-dir>
+ <expected-error>ASX1154: Incompatible index type ARRAY</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/error-handling">
+ <compilation-unit name="index-mixed-composite">
+ <output-dir compare="Text">index-mixed-composite</output-dir>
+ <expected-error>ASX1154: Incompatible index type ARRAY</expected-error>
+ <source-location>false</source-location>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/metadata">
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <!-- <test-case FilePath="array-index/metadata/closed">-->
+ <!-- <compilation-unit name="with-composite-sk">-->
+ <!-- <output-dir compare="Text">with-composite-sk</output-dir>-->
+ <!-- </compilation-unit>-->
+ <!-- </test-case>-->
+ <!-- <test-case FilePath="array-index/metadata/open">-->
+ <!-- <compilation-unit name="with-composite-sk">-->
+ <!-- <output-dir compare="Text">with-composite-sk</output-dir>-->
+ <!-- </compilation-unit>-->
+ <!-- </test-case>-->
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-composite-array-different-indicators">
+ <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-composite-array-different-indicators">
+ <output-dir compare="Text">with-composite-array-different-indicators</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/metadata/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/bulk-loading/on-index-creation">
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/on-index-creation/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/bulk-loading/after-index-creation">
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/bulk-loading/after-index-creation">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/insert-upsert-delete">
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-additional-atomic-index">
+ <output-dir compare="Text">with-additional-atomic-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/open">
+ <compilation-unit name="with-additional-atomic-index">
+ <output-dir compare="Text">with-additional-atomic-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/insert-upsert-delete/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/join-unnest-queries">
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/join-unnest-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/select-unnest-queries">
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-3-level-record-path">
+ <output-dir compare="Text">with-3-level-record-path</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/open">
+ <compilation-unit name="with-composite-sk">
+ <output-dir compare="Text">with-composite-sk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-unnest-queries/closed">
+ <compilation-unit name="with-filter-fields">
+ <output-dir compare="Text">with-filter-fields</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="array-index/select-quantified-queries">
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-1">
+ <output-dir compare="Text">use-case-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-2">
+ <output-dir compare="Text">use-case-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-3">
+ <output-dir compare="Text">use-case-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="use-case-4">
+ <output-dir compare="Text">use-case-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-composite-pk">
+ <output-dir compare="Text">with-composite-pk</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="array-index/select-quantified-queries">
+ <compilation-unit name="with-open-index">
+ <output-dir compare="Text">with-open-index</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ </test-group>
<test-group name="nestrecords">
<test-case FilePath="nestrecords">
<compilation-unit name="nestrecord">
@@ -12693,7 +13108,7 @@
<test-case FilePath="feeds">
<compilation-unit name="change-feed-with-meta-with-mixed-index">
<output-dir compare="Text">change-feed-with-meta-with-mixed-index</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 1)</expected-error>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 61, at column 35)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
@@ -12709,7 +13124,7 @@
<test-case FilePath="feeds">
<compilation-unit name="change-feed-with-meta-open-index-in-meta">
<output-dir compare="Text">change-feed-with-meta-open-index-in-meta</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 34)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
@@ -12720,13 +13135,13 @@
<test-case FilePath="feeds">
<compilation-unit name="change-feed-with-meta-pk-in-meta-index-after-ingest">
<output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-after-ingest</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 1)</expected-error>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 27, at column 37)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
<compilation-unit name="change-feed-with-meta-pk-in-meta-index-in-meta">
<output-dir compare="Text">change-feed-with-meta-pk-in-meta-index-in-meta</output-dir>
- <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 1)</expected-error>
+ <expected-error>Compilation error: Cannot create index on meta fields (in line 60, at column 37)</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="feeds">
diff --git a/asterixdb/asterix-benchmark/pom.xml b/asterixdb/asterix-benchmark/pom.xml
index 8aac937..6801276 100644
--- a/asterixdb/asterix-benchmark/pom.xml
+++ b/asterixdb/asterix-benchmark/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-benchmark</artifactId>
diff --git a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q11.sqlpp b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q11.sqlpp
index a6c0308..b3673f2 100644
--- a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q11.sqlpp
+++ b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q11.sqlpp
@@ -32,7 +32,7 @@
)[0]
-SELECT ps_partkey, SUM(ps.ps_supplycost * ps.ps_availqty) AS part_value
+SELECT ps_partkey, part_value
FROM Partsupp ps,
(
SELECT s.s_suppkey
@@ -42,6 +42,7 @@
) sn
WHERE ps.ps_suppkey = sn.s_suppkey
GROUP BY ps.ps_partkey
+LET part_value = SUM(ps.ps_supplycost * ps.ps_availqty)
HAVING part_value > sum * 0.0001000
ORDER BY part_value DESC
;
diff --git a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q20.sqlpp.dis b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q20.sqlpp.dis
index 45c4740..230c19a 100644
--- a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q20.sqlpp.dis
+++ b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q20.sqlpp.dis
@@ -38,12 +38,12 @@
,
q20_tmp3 AS
(
- SELECT ps_suppkey, ps_availqty, t2.sum_quantity
- FROM Partsupp
+ SELECT ps.ps_suppkey, ps.ps_availqty, t2.sum_quantity
+ FROM Partsupp ps
JOIN q20_tmp1 t1
- ON ps_partkey = t1.p_partkey
+ ON ps.ps_partkey = t1.p_partkey
JOIN q20_tmp2 t2
- ON ps_partkey = t2.l_partkey and ps_suppkey = t2.l_suppkey
+ ON ps.ps_partkey = t2.l_partkey and ps.ps_suppkey = t2.l_suppkey
)
,
q20_tmp4 AS
diff --git a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q21.sqlpp.dis b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q21.sqlpp.dis
index 768ad47..6e46f8c 100644
--- a/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q21.sqlpp.dis
+++ b/asterixdb/asterix-benchmark/src/main/resources/benchmarks/tpch/queries/q21.sqlpp.dis
@@ -35,7 +35,7 @@
),
tmp2 AS
(
- SELECT l2.l_orderkey,
+ SELECT l_orderkey,
COUNT(l_suppkey) AS count_suppkey,
MAX(l_suppkey) AS max_suppkey
FROM (
@@ -49,7 +49,7 @@
SELECT t4.s_name, COUNT(*) AS numwait
FROM (
- SELECT t3.s_name, l_suppkey, t2.l_orderkey, count_suppkey, max_suppkey
+ SELECT t3.s_name, t3.l_suppkey, t2.l_orderkey, t3.count_suppkey, t3.max_suppkey
FROM (
SELECT ns.s_name, t1.l_orderkey, t1.l_suppkey
FROM LineItem l,
@@ -64,7 +64,7 @@
AND o.o_orderkey = t1.l_orderkey AND l.l_orderkey = t1.l_orderkey
AND o.o_orderstatus = 'F'
) AS t3
- JOIN tmp2 AS t2 ON count_suppkey >= 0 AND t3.l_orderkey = t2.l_orderkey
+ JOIN tmp2 AS t2 ON t3.count_suppkey >= 0 AND t3.l_orderkey = t2.l_orderkey
) AS t4
GROUP BY t4.s_name
ORDER BY numwait DESC, t4.s_name
diff --git a/asterixdb/asterix-client-helper/pom.xml b/asterixdb/asterix-client-helper/pom.xml
index 403360b..4f3bfb6 100644
--- a/asterixdb/asterix-client-helper/pom.xml
+++ b/asterixdb/asterix-client-helper/pom.xml
@@ -22,7 +22,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<properties>
<root.dir>${basedir}/..</root.dir>
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 2d8bec2..c103913 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-common</artifactId>
<licenses>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
index c85d091..79f13bf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IMetadataLockManager.java
@@ -155,6 +155,66 @@
throws AlgebricksException;
/**
+ * Acquire read lock on the full-text config
+ *
+ * @param locks
+ * the lock list to add the new lock to
+ * @param dataverseName
+ * the dataverse name
+ * @param fullTextConfigName
+ * the name of the full-text config in the given dataverse
+ * @throws AlgebricksException
+ * if lock couldn't be acquired
+ */
+ void acquireFullTextConfigReadLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+ throws AlgebricksException;
+
+ /**
+ * Acquire write lock on the full-text config
+ *
+ * @param locks
+ * the lock list to add the new lock to
+ * @param dataverseName
+ * the dataverse name
+ * @param fullTextConfigName
+ * the name of the full-text config in the given dataverse
+ * @throws AlgebricksException
+ * if lock couldn't be acquired
+ */
+ void acquireFullTextConfigWriteLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+ throws AlgebricksException;
+
+ /**
+ * Acquire read lock on the full-text filter
+ *
+ * @param locks
+ * the lock list to add the new lock to
+ * @param dataverseName
+ * the dataverse name
+ * @param fullTextFilterName
+ * the name of the full-text filter in the given dataverse
+ * @throws AlgebricksException
+ * if lock couldn't be acquired
+ */
+ void acquireFullTextFilterReadLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+ throws AlgebricksException;
+
+ /**
+ * Acquire write lock on the full-text filter
+ *
+ * @param locks
+ * the lock list to add the new lock to
+ * @param dataverseName
+ * the dataverse name
+ * @param fullTextFilterName
+ * the name of the full-text filter in the given dataverse
+ * @throws AlgebricksException
+ * if lock couldn't be acquired
+ */
+ void acquireFullTextFilterWriteLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+ throws AlgebricksException;
+
+ /**
* Acquire read lock on the library
*
* @param locks
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index a58673f..5538a0a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -99,7 +99,11 @@
COMPILER_MIN_MEMORY_ALLOCATION(
BOOLEAN,
AlgebricksConfig.MIN_MEMORY_ALLOCATION_DEFAULT,
- "Enable/disable allocating minimum budget for certain queries");
+ "Enable/disable allocating minimum budget for certain queries"),
+ COMPILER_ARRAYINDEX(
+ BOOLEAN,
+ AlgebricksConfig.ARRAY_INDEX_DEFAULT,
+ "Enable/disable using array-indexes in queries");
private final IOptionType type;
private final Object defaultValue;
@@ -165,6 +169,8 @@
public static final String COMPILER_MIN_MEMORY_ALLOCATION_KEY = Option.COMPILER_MIN_MEMORY_ALLOCATION.ini();
+ public static final String COMPILER_ARRAYINDEX_KEY = Option.COMPILER_ARRAYINDEX.ini();
+
public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
@@ -233,6 +239,10 @@
return accessor.getBoolean(Option.COMPILER_MIN_MEMORY_ALLOCATION);
}
+ public boolean isArrayIndex() {
+ return accessor.getBoolean(Option.COMPILER_ARRAYINDEX);
+ }
+
public int getExternalScanMemorySize() {
return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
index 3c9acf4..f673d24 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/DatasetConfig.java
@@ -39,7 +39,8 @@
SINGLE_PARTITION_WORD_INVIX,
SINGLE_PARTITION_NGRAM_INVIX,
LENGTH_PARTITIONED_WORD_INVIX,
- LENGTH_PARTITIONED_NGRAM_INVIX
+ LENGTH_PARTITIONED_NGRAM_INVIX,
+ ARRAY;
}
public enum TransactionState {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index c4dbd85..d498d69 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -72,9 +72,12 @@
CompilerProperties.COMPILER_SUBPLAN_NESTEDPUSHDOWN_KEY, compilerProperties.getSubplanNestedPushdown());
boolean minMemoryAllocation = getBoolean(querySpecificConfig,
CompilerProperties.COMPILER_MIN_MEMORY_ALLOCATION_KEY, compilerProperties.getMinMemoryAllocation());
+ boolean arrayIndex = getBoolean(querySpecificConfig, CompilerProperties.COMPILER_ARRAYINDEX_KEY,
+ compilerProperties.isArrayIndex());
int externalScanBufferSize = getExternalScanBufferSize(
(String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
compilerProperties.getExternalScanMemorySize(), sourceLoc);
+
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
physOptConf.setMaxFramesExternalSort(sortFrameLimit);
@@ -90,6 +93,7 @@
physOptConf.setSubplanMerge(subplanMerge);
physOptConf.setSubplanNestedPushdown(subplanNestedPushdown);
physOptConf.setMinMemoryAllocation(minMemoryAllocation);
+ physOptConf.setArrayIndexEnabled(arrayIndex);
physOptConf.setExternalScanBufferSize(externalScanBufferSize);
return physOptConf;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 4078ea2..c87cb7f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -235,6 +235,12 @@
CANNOT_DROP_OBJECT_DEPENDENT_EXISTS(1148),
ILLEGAL_FUNCTION_RECURSION(1149),
ILLEGAL_FUNCTION_USE(1150),
+ FULL_TEXT_CONFIG_NOT_FOUND(1151),
+ FULL_TEXT_FILTER_NOT_FOUND(1152),
+ FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED(1153),
+ COMPILATION_INCOMPATIBLE_INDEX_TYPE(1154),
+ FULL_TEXT_CONFIG_ALREADY_EXISTS(1155),
+ FULL_TEXT_FILTER_ALREADY_EXISTS(1156),
// Feed errors
DATAFLOW_ILLEGAL_STATE(3001),
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
index a0081cb..28eb553 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/IMetadataLockUtil.java
@@ -23,6 +23,8 @@
import org.apache.asterix.common.config.DatasetConfig;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import com.google.common.collect.ImmutableList;
+
public interface IMetadataLockUtil {
// Dataverse helpers
@@ -59,7 +61,7 @@
// Index helpers
void createIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
- String datasetName) throws AlgebricksException;
+ String datasetName, String fullTextConfigName) throws AlgebricksException;
void dropIndexBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
String datasetName) throws AlgebricksException;
@@ -72,7 +74,7 @@
void dropTypeBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName, String typeName)
throws AlgebricksException;
- // Function helpers
+ // Library helpers
void createLibraryBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
String libraryName) throws AlgebricksException;
@@ -88,6 +90,22 @@
void dropFunctionBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
String functionName) throws AlgebricksException;
+ // Full-text filter helpers
+
+ void createFullTextFilterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+ String fullTextFilterName) throws AlgebricksException;
+
+ void dropFullTextFilterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+ String fullTextFilterName) throws AlgebricksException;
+
+ // Full-text config helpers
+
+ void createFullTextConfigBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+ String fullTextConfigName, ImmutableList<String> fullTextFilterNames) throws AlgebricksException;
+
+ void dropFullTextConfigBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
+ String fullTextFilterName) throws AlgebricksException;
+
// Adapter helpers
void createAdapterBegin(IMetadataLockManager lockManager, LockList locks, DataverseName dataverseName,
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 18898bb..f7c6a77 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -237,6 +237,12 @@
1148 = Cannot drop %1$s %2$s being used by %3$s %4$s
1149 = Illegal function recursion
1150 = Illegal use of function %1$s
+1151 = Full-text config %1$s not found
+1152 = Full-text filter %1$s not found
+1153 = Default full-text config with a name of null cannot be deleted or created
+1154 = Incompatible index type %1$s
+1155 = Full-text config %1$s already exists
+1156 = Full-text filter %1$s already exists
# Feed Errors
3001 = Illegal state.
diff --git a/asterixdb/asterix-coverage/pom.xml b/asterixdb/asterix-coverage/pom.xml
index a3de43d..716f200 100644
--- a/asterixdb/asterix-coverage/pom.xml
+++ b/asterixdb/asterix-coverage/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index b8ac630..8bf3389 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<artifactId>asterix-dashboard</artifactId>
@@ -68,8 +68,8 @@
<artifactId>frontend-maven-plugin</artifactId>
<version>1.6</version>
<configuration>
- <nodeVersion>v10.3.0</nodeVersion>
- <npmVersion>6.1.0</npmVersion>
+ <nodeVersion>v14.15.4</nodeVersion>
+ <npmVersion>6.14.11</npmVersion>
<workingDirectory>target/dashboard</workingDirectory>
<installDirectory>target/dashboard</installDirectory>
</configuration>
diff --git a/asterixdb/asterix-dashboard/src/node/angular.json b/asterixdb/asterix-dashboard/src/node/angular.json
index 768339e..839ff38 100644
--- a/asterixdb/asterix-dashboard/src/node/angular.json
+++ b/asterixdb/asterix-dashboard/src/node/angular.json
@@ -28,6 +28,11 @@
],
"scripts": [
"node_modules/codemirror/lib/codemirror.js"
+ ],
+ "allowedCommonJsDependencies": [
+ "hammerjs",
+ "lodash",
+ "file-saver",
]
},
"configurations": {
@@ -57,7 +62,8 @@
},
"configurations": {
"production": {
- "browserTarget": "asterixdb-web-console:build:production"
+ "browserTarget": "asterixdb-web-console:build:production",
+ "sourceMap": true
}
}
},
@@ -140,4 +146,4 @@
"prefix": "app"
}
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/package.json b/asterixdb/asterix-dashboard/src/node/package.json
index c716289..da627b9 100755
--- a/asterixdb/asterix-dashboard/src/node/package.json
+++ b/asterixdb/asterix-dashboard/src/node/package.json
@@ -13,52 +13,53 @@
},
"private": true,
"dependencies": {
- "@angular-devkit/core": "0.6.8",
- "@angular/animations": "6.0.7",
- "@angular/cdk": "6.3.3",
- "@angular/common": "6.0.7",
- "@angular/compiler": "6.0.7",
- "@angular/core": "6.0.7",
- "@angular/forms": "6.0.7",
- "@angular/http": "6.0.7",
- "@angular/material": "6.3.3",
- "@angular/platform-browser": "6.0.7",
- "@angular/platform-browser-dynamic": "6.0.7",
- "@angular/router": "6.0.7",
- "@ngrx/effects": "6.0.0",
- "@ngrx/entity": "6.0.0",
- "@ngrx/store": "6.0.0",
- "@ngrx/store-devtools": "6.0.0",
- "codemirror": "5.31.0",
- "core-js": "2.4.1",
- "file-saver": "1.3.3",
+ "@angular-devkit/core": "10.2.1",
+ "@angular/animations": "10.2.4",
+ "@angular/cdk": "10.2.4",
+ "@angular/common": "10.2.4",
+ "@angular/compiler": "10.2.4",
+ "@angular/core": "10.2.4",
+ "@angular/forms": "10.2.4",
+ "@angular/material": "10.2.4",
+ "@angular/platform-browser": "10.2.4",
+ "@angular/platform-browser-dynamic": "10.2.4",
+ "@angular/router": "10.2.4",
+ "@ngrx/effects": "10.1.2",
+ "@ngrx/entity": "10.1.2",
+ "@ngrx/store": "10.1.2",
+ "@ngrx/store-devtools": "10.1.2",
+ "@swimlane/ngx-charts": "^16.0.0",
+ "@swimlane/ngx-graph": "^7.2.0",
+ "codemirror": "5.59.2",
+ "core-js": "3.8.3",
+ "file-saver": "2.0.5",
"hammerjs": "2.0.8",
- "lodash": "^4.17.10",
+ "lodash": "^4.17.20",
"roboto-fontface": "^0.10.0",
- "rxjs": "6.2.1",
- "rxjs-compat": "6.2.1",
- "zone.js": "0.8.26"
+ "rxjs": "6.6.3",
+ "rxjs-compat": "6.6.3",
+ "zone.js": "0.10.3"
},
"devDependencies": {
- "@angular/cli": "6.0.8",
- "@angular/compiler-cli": "6.0.7",
- "@angular/language-service": "6.0.7",
- "@types/file-saver": "1.3.0",
- "@types/jasmine": "2.5.53",
- "@types/jasminewd2": "2.0.2",
- "@types/node": "6.0.60",
- "jasmine-core": "2.6.2",
- "jasmine-spec-reporter": "4.1.0",
- "karma": "1.7.0",
- "karma-chrome-launcher": "2.1.1",
- "karma-cli": "1.0.1",
- "karma-coverage-istanbul-reporter": "1.2.1",
- "karma-jasmine": "1.1.0",
- "karma-jasmine-html-reporter": "0.2.2",
- "protractor": "5.1.2",
- "ts-node": "3.2.2",
- "tslint": "5.7.0",
- "typescript": "2.7.2",
- "@angular-devkit/build-angular": "0.6.8"
+ "@angular-devkit/build-angular": "0.1002.1",
+ "@angular/cli": "10.2.1",
+ "@angular/compiler-cli": "10.2.4",
+ "@angular/language-service": "10.2.4",
+ "@types/file-saver": "2.0.1",
+ "@types/jasmine": "3.6.3",
+ "@types/jasminewd2": "2.0.8",
+ "@types/node": "14.14.22",
+ "eslint": "^7.1.0",
+ "jasmine-core": "3.6.0",
+ "jasmine-spec-reporter": "5.0.2",
+ "karma": "5.0.9",
+ "karma-chrome-launcher": "3.1.0",
+ "karma-cli": "2.0.0",
+ "karma-coverage-istanbul-reporter": "3.0.2",
+ "karma-jasmine": "3.2.0",
+ "karma-jasmine-html-reporter": "1.5.4",
+ "protractor": "7.0.0",
+ "ts-node": "8.10.1",
+ "typescript": "4.0.5"
}
}
diff --git a/asterixdb/asterix-dashboard/src/node/proxy.config.js b/asterixdb/asterix-dashboard/src/node/proxy.config.js
index 58752c3..22fcc71 100755
--- a/asterixdb/asterix-dashboard/src/node/proxy.config.js
+++ b/asterixdb/asterix-dashboard/src/node/proxy.config.js
@@ -17,7 +17,12 @@
"secure": false,
logLevel: "debug",
pathRewrite: function (path) { return path.replace('/query-service', '/query/service')}
+ },
+ "/admin/requests/running": {
+ "target": "http://localhost:19002",
+ "secure": false,
+ logLevel: "debug"
}
}
-module.exports = PROXY_CONFIG;
\ No newline at end of file
+module.exports = PROXY_CONFIG;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/app-config.service.ts b/asterixdb/asterix-dashboard/src/node/src/app/app-config.service.ts
index 5227227..0c86099 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/app-config.service.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/app-config.service.ts
@@ -1,5 +1,5 @@
-import {throwError as observableThrowError, Observable } from 'rxjs';
+import {throwError as observableThrowError } from 'rxjs';
/*
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
@@ -14,40 +14,54 @@
limitations under the License.
*/
import { Injectable } from '@angular/core';
-import { Http, Headers, RequestOptions } from '@angular/http';
+import { HttpClient, HttpHeaders } from '@angular/common/http';
import 'rxjs/add/operator/map';
import 'rxjs/add/operator/catch';
+interface ENV_DATA {
+ env: string
+}
+
@Injectable()
export class ConfigService {
private config: Object
private env: Object
- constructor(private http: Http) {}
+ constructor(private http: HttpClient) {}
+
/**
* Loads the environment config file first. Reads the environment variable from the file
* and based on that loads the appropriate configuration file - development or production
*/
load() {
return new Promise((resolve, reject) => {
- let headers = new Headers({ 'Accept': 'application/json', 'Content-Type': 'application/json', 'DataType': 'application/json' });
- let options = new RequestOptions({ headers: headers });
- this.http.get('/config/env.json')
- .map(res => res.json())
- .subscribe((env_data) => {
- this.env = env_data;
+ let headers = new HttpHeaders({ 'Accept': 'application/json', 'Content-Type': 'application/json', 'DataType': 'application/json' });
+ const HttpOptions = {
+ headers: new HttpHeaders({
+ 'Accept': 'application/json',
+ 'Content-Type': 'application/json',
+ 'DataType': 'application/json',
+ })
+ }
- this.http.get('/config/' + env_data.env + '.json')
- .map(res => res.json())
+ this.http.get('/config/env.json', HttpOptions)
+ .subscribe({
+ next(env_data: ENV_DATA) {
+ this.env = env_data;
+
+ this.http.get('/config/' + env_data.env + '.json')
.catch((error: any) => {
- return observableThrowError(error.json().error || 'Server error');
- })
- .subscribe((data) => {
- this.config = data;
- resolve(true);
- });
- });
+ return observableThrowError(error.json().error || 'Server error');
+ })
+ .subscribe({
+ next(data) {
+ this.config = data;
+ resolve(true);
+ }
+ })
+ }
+ });
});
}
@@ -68,4 +82,4 @@
get(key: any) {
return this.config[key];
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/app.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/app.component.ts
index 6cd84a9..bad2140 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/app.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/app.component.ts
@@ -27,4 +27,4 @@
title = 'Asterix DB Web Console';
constructor() {}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/app.module.ts b/asterixdb/asterix-dashboard/src/node/src/app/app.module.ts
index e49d8e2..94d9a05 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/app.module.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/app.module.ts
@@ -11,7 +11,7 @@
See the License for the specific language governing permissions and
limitations under the License.
*/
-import { NgModule } from '@angular/core';
+import { NgModule, CUSTOM_ELEMENTS_SCHEMA } from '@angular/core';
import { AppComponent } from './app.component';
import { AppEffects } from './shared/effects/app.effects';
import { BrowserModule } from '@angular/platform-browser';
@@ -22,6 +22,7 @@
import { DatasetEffects } from './shared/effects/dataset.effects';
import { DatatypeEffects } from './shared/effects/datatype.effects';
import { IndexEffects } from './shared/effects/index.effects';
+import { FunctionEffects } from "./shared/effects/function.effects";
import { SQLQueryEffects } from './shared/effects/query.effects';
import { AppBarComponent } from './dashboard/appbar.component';
import { DialogMetadataInspector, MetadataComponent } from './dashboard/query/metadata.component';
@@ -35,10 +36,12 @@
import { MaterialModule } from './material.module';
import { StoreModule, } from '@ngrx/store';
import { StoreDevtoolsModule } from '@ngrx/store-devtools';
-import { PlanViewComponent } from './dashboard/query/plan-view.component';
-import { PlanNodeSVGComponent } from './dashboard/query/plan-node-svg.component';
+import { NgxGraphModule } from "@swimlane/ngx-graph";
+import { NgxChartsModule } from "@swimlane/ngx-charts";
+import { PlanViewerComponent } from "./dashboard/query/plan-viewer.component";
import { TreeNodeComponent } from './dashboard/query/tree-node.component';
-import { TreeViewComponent } from './dashboard/query/tree-view.component';
+import { DialogExportPicker, TreeViewComponent } from './dashboard/query/tree-view.component';
+import {SQLCancelEffects} from "./shared/effects/cancel.effects";
@NgModule({
declarations: [
@@ -50,27 +53,38 @@
QueryContainerComponent,
AppTabComponent,
DialogMetadataInspector,
- PlanNodeSVGComponent,
- PlanViewComponent,
+ PlanViewerComponent,
TreeNodeComponent,
+ DialogExportPicker,
TreeViewComponent,
],
imports: [
FormsModule,
BrowserModule,
BrowserAnimationsModule,
- EffectsModule.forRoot([AppEffects, DataverseEffects, DatasetEffects, DatatypeEffects, IndexEffects, SQLQueryEffects]),
+ EffectsModule.forRoot([AppEffects, DataverseEffects, DatasetEffects, DatatypeEffects, IndexEffects, FunctionEffects, SQLQueryEffects, SQLCancelEffects]),
HttpClientModule,
MaterialModule,
- StoreModule.forRoot(reducers),
+ StoreModule.forRoot(reducers, {
+ runtimeChecks: {
+ strictStateImmutability: false,
+ strictActionImmutability: false,
+ strictStateSerializability: false,
+ strictActionSerializability: false,
+ },
+ }),
StoreDevtoolsModule.instrument({
maxAge: 10
- })
+ }),
+ NgxGraphModule,
+ NgxChartsModule
],
+ schemas: [CUSTOM_ELEMENTS_SCHEMA],
entryComponents: [
- DialogMetadataInspector
+ DialogMetadataInspector,
+ DialogExportPicker,
],
providers: [SQLService],
bootstrap: [AppComponent]
})
-export class AppModule {}
\ No newline at end of file
+export class AppModule {}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.html
index 45e3008..3c3d1f3 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.html
@@ -18,24 +18,21 @@
<div class="flex-spacer"></div>
<nav class="awc-navbar-header">
<div class="menu">
- <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://asterixDB.apache.org" aria-label="WEBSITE">
+ <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://asterixdb.apache.org/" target="_blank" aria-label="WEBSITE">
WEBSITE
</a>
- <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://issues.apache.org/jira/browse/ASTERIXDB" aria-label="FILE ISSUES">
+ <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://issues.apache.org/jira/browse/ASTERIXDB" target="_blank" aria-label="FILE ISSUES">
FILE ISSUES
</a>
- <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://ci.apache.org/projects/asterixdb/index.html" aria-label="DOCUMENTATION">
+ <a mat-button class="menu awc-button awc-navbar-hide-small" href="https://ci.apache.org/projects/asterixdb/index.html" target="_blank" aria-label="DOCUMENTATION">
DOCUMENTATION
</a>
- <a mat-button class="menu awc-button docs-navbar-hide-small" href="https://asterixdb.apache.org/community.html" aria-label="CONTACT">
+ <a mat-button class="menu awc-button docs-navbar-hide-small" href="https://asterixdb.apache.org/community.html" target="_blank" aria-label="CONTACT">
CONTACT
</a>
- <a mat-button class="menu awc-button docs-navbar-hide-small" href="https://github.com/apache/asterixdb/" aria-label="GITHUB">
+ <a mat-button class="menu awc-button docs-navbar-hide-small" href="https://github.com/apache/asterixdb/" target="_blank" aria-label="GITHUB">
GITHUB
</a>
- <a mat-button class="menu awc-button docs-navbar-hide-small" aria-label="METADATA" (click)='showMetadata()'>
- METADATA
- </a>
</div>
</nav>
-</header>
\ No newline at end of file
+</header>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.scss
index 288bdb9..c16a172 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.scss
@@ -61,7 +61,7 @@
}
.menu {
- /deep/ .mat-tab-label {
+ >>> .mat-tab-label {
font-size: 0.80rem !important;
font-weight: 500 !important;
}
@@ -71,4 +71,4 @@
margin: 0;
margin-right: 15px;
margin-left: 5px;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.ts
index c399883..7538bf2 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/appbar.component.ts
@@ -23,12 +23,5 @@
})
export class AppBarComponent {
- sideMenuVisible = false;
-
constructor(private store: Store <any> ) {}
-
- showMetadata() {
- this.sideMenuVisible = !this.sideMenuVisible;
- this.store.dispatch(new appActions.setSideMenuVisible(this.sideMenuVisible));
- }
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/apptab.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/apptab.component.scss
index c14f327..1745fd6 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/apptab.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/apptab.component.scss
@@ -29,10 +29,10 @@
}
.menu {
- /deep/ .mat-tab-label {
+ >>> .mat-tab-label {
font-size: 0.80rem !important;
font-weight: 500 !important;
color: white;
}
background-color: blue;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.html
new file mode 100644
index 0000000..d12b18b
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.html
@@ -0,0 +1,33 @@
+<!--/*
+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.
+*/ -->
+<div class="export-header">
+ <pre class="export-title">Choose Export Format:</pre>
+</div>
+<div class="button-row" *ngIf="data.exportFormat == 'json' || data.exportFormat == 'jsonl'">
+ <button mat-button class="pick-btn" (click)="data.exportFormat = 'json'">JSON</button>
+ <button mat-button class="pick-btn" (click)="data.exportFormat = 'jsonl'">JSONL</button>
+</div>
+<div class="filename-input">
+ <mat-form-field>
+ <mat-label>Filename:</mat-label>
+ <input matInput [(ngModel)]="data.fileName" value="asterix-query-results">
+ </mat-form-field>
+ <pre *ngIf="data.exportFormat == 'json'">.json</pre>
+ <pre *ngIf="data.exportFormat == 'jsonl'">.jsonl</pre>
+ <pre *ngIf="data.exportFormat == 'csv'">.csv</pre>
+</div>
+<div class="cancel-row">
+ <button mat-button class="cancel-btn btn" [mat-dialog-close]="['cancel', data.fileName]">CANCEL</button>
+ <button mat-button class="export-btn btn" [mat-dialog-close]="[data.exportFormat, data.fileName]">EXPORT</button>
+</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.scss
new file mode 100644
index 0000000..456b260
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/dialog-export-picker.scss
@@ -0,0 +1,53 @@
+/*
+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.
+*/
+
+.export-header {
+ border-bottom: 1px dashed gray;
+ text-align: center;
+}
+
+.export-title {
+ color: blue;
+}
+
+.button-row {
+ padding-top: 20px;
+ display: flex;
+ flex-flow: row;
+ justify-content: space-between;
+ border-bottom: 1px dashed gray;
+ padding-bottom: 20px;
+}
+
+.filename-input {
+ padding-top: 20px;
+ display: flex;
+ flex-flow: row;
+ border-bottom: 1px dashed gray;
+ padding-bottom: 20px;
+}
+
+.cancel-row {
+ display: flex;
+ flex-flow: row;
+}
+
+.cancel-btn {
+ margin-left: auto;
+}
+
+.pick-btn {
+ border: 2px solid blue;
+ border-radius: 5px;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.html
index d410ff1..549615f 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.html
@@ -14,7 +14,7 @@
<mat-expansion-panel class="card" hideToggle [expanded]="true">
<mat-expansion-panel-header class="header" >
<mat-panel-title>
- <mat-panel-title>SQL++ INPUT ({{currentQuery+1}}/{{preparedQueryCount}})
+ <mat-panel-title>QUERY INPUT ({{currentQuery+1}}/{{preparedQueryCount}})
</mat-panel-title>
<mat-panel-description></mat-panel-description>
<mat-spinner *ngIf="querySpinnerVisible" [color]="blue" [diameter]="15" class="spinner"></mat-spinner>
@@ -25,8 +25,7 @@
<div class='dataverses'>
<div class='d1'>
<mat-form-field>
- <mat-select placeholder="USE DATAVERSE" [(ngModel)]="selected" (selectionChange)="dataverseSelected()">
- <mat-option value='None'>None</mat-option>
+ <mat-select placeholder="Default" [(ngModel)]="selected" (selectionChange)="dataverseSelected()">
<mat-option *ngFor="let dataverse of dataverses" [value]="dataverse.DataverseName">
{{dataverse.DataverseName}}
</mat-option>
@@ -34,20 +33,35 @@
</mat-form-field>
</div>
<div class='d1'>
- <mat-form-field>
- <mat-select placeholder="PLAN FORMAT" [(ngModel)]="formatOptions">
+ <mat-form-field class="plan-output-format">
+ <mat-select id="plan-format" placeholder="PLAN FORMAT" [(ngModel)]="formatOptions">
<mat-option value="JSON">JSON</mat-option>
<mat-option value="STRING">STRING</mat-option>
</mat-select>
</mat-form-field>
</div>
+ <div class="d1">
+ <mat-form-field class="plan-output-format">
+ <mat-select class="mat-select output-format" placeholder="OUTPUT FORMAT" [(ngModel)]="outputOptions">
+ <mat-option value="JSON">JSON</mat-option>
+ <mat-option id="csv-option" value="CSV">CSV (no header)</mat-option>
+ <mat-option value="CSV_header">CSV (header)</mat-option>
+ </mat-select>
+ </mat-form-field>
+ </div>
<div class='d1'>
<mat-form-field class='sql-history'>
- <mat-select placeholder="SQL++ HISTORY" [(ngModel)]="historyStringSelected" (selectionChange)="historySelected()">
- <mat-option *ngFor="let query of history" [value]="query">{{query}}</mat-option>
+ <mat-select placeholder="QUERY HISTORY" [(ngModel)]="historyIdxSelected" (ngModelChange)="historySelected($event)">
+ <mat-option *ngFor="let query of history" [value]="query.index">
+ {{query.query}}
+ </mat-option>
</mat-select>
</mat-form-field>
</div>
+ <div class="space">
+ <button mat-icon-button class='input-button next-prev-btn' (click)="onClickNext()" [disabled]="checkNext()" matTooltip="Next Query Input"><mat-icon>keyboard_arrow_right</mat-icon></button>
+ <button mat-icon-button class='input-button next-prev-btn' (click)="onClickPrevious()" [disabled]="checkPrevious()" matTooltip="Previous Query Input"><mat-icon>keyboard_arrow_left</mat-icon></button>
+ </div>
</div>
</div>
<div class="codemirror-container">
@@ -55,15 +69,33 @@
</div>
</mat-panel-description>
<div class="message">
- <span *ngIf="querySuccess" class="metrics">{{metricsString}}</span>
+ <span *ngIf="querySuccess==true && queryWarningsShow==false" class="metrics">{{metricsString}}</span>
<span *ngIf="queryError" class="queryErrorMessage">{{queryErrorMessageString}}</span>
+ <span *ngIf="querySuccess==true && queryWarningsShow==true" class="queryWarningMessage" matTooltip="{{queryWarningsMessages}}">{{metricsString}}</span>
+ <mat-divider></mat-divider>
+ <div class="obj-returned-div">
+ <span *ngIf="querySuccess" class="obj-returned-cnt">Objects Returned: {{objectsReturned}}</span>
+ </div>
+ <mat-divider *ngIf="querySuccess"></mat-divider>
+ <mat-expansion-panel [disabled]="queryWarningsCount == 0" class="tree-node mat-elevation-z0">
+ <mat-expansion-panel-header>
+ <div>
+ <b>WARNINGS({{queryWarningsCount}})</b>
+ </div>
+ </mat-expansion-panel-header>
+ <mat-panel-description class="content">
+ <div class="warning-msgs">
+ <pre *ngFor="let warning of queryWarningsMessages">{{warning}}</pre>
+ </div>
+ </mat-panel-description>
+ </mat-expansion-panel>
</div>
<mat-action-row>
+ <button mat-button class='input-button clear' (click)="onClickClear()" matTooltip="Clear Query Input">CLEAR</button>
<div class="space"></div>
- <button mat-button class='input-button' (click)="onClickNew()" matTooltip="New Query Input">NEW</button>
- <button mat-button class='input-button' (click)="onClickClear()" matTooltip="Clear Query Input">CLEAR</button>
- <button mat-button class='input-button run' (click)="onClickRun()" matTooltip="Execute Query Input">RUN</button>
- <button mat-button class='input-button' (click)="onClickPrevious()" [disabled]="checkPrevious()" matTooltip="Previous Query Input">PREVIOUS</button>
- <button mat-button class='input-button' (click)="onClickNext()" [disabled]="checkNext()" matTooltip="Next Query Input">NEXT</button>
+ <button mat-button class="input-button explain" (click)="onClickExplain()" matTooltip="Explain Query">EXPLAIN</button>
+ <!--<button mat-button class='input-button run' (click)="onClickRun()" matTooltip="Execute Query Input">EXECUTE</button>-->
+ <button mat-icon-button class='input-button stop' (click)="onClickStop()" [disabled]="!querySpinnerVisible" matTooltip="Cancel Query"><mat-icon>stop</mat-icon></button>
+ <button mat-icon-button class='input-button run' (click)="onClickRun()" matTooltip="Execute Query"><mat-icon>play_arrow</mat-icon></button>
</mat-action-row>
-</mat-expansion-panel>
\ No newline at end of file
+</mat-expansion-panel>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
index aa66fb9..94311f8 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.scss
@@ -58,16 +58,30 @@
border: 1px dashed gainsboro;
}
+#clear-history-btn {
+ color: black;
+}
+
.input-button {
font-size: 12px !important;
}
+.next-prev-btn {
+ color: black;
+ float: right;
+}
+
+.next-prev-btn[disabled] {
+ color: rgba(0, 0, 0, 0.26);
+}
+
+
.message {
margin-top: 30px;
}
.metrics {
- color: blue;
+ color: green;
font-size: 14px !important;
font-size: 1.0rem;
font-weight: 500;
@@ -89,8 +103,39 @@
padding: 0;
}
+.queryWarningMessage {
+ color: #ffae42;
+ font-size: 14px !important;
+ word-break: break-all;
+ font-size: 1.0rem;
+ font-weight: 500;
+ margin: 0;
+ padding: 0;
+}
+
+.obj-returned-cnt {
+ font-size: 14px !important;
+ word-break: break-all;
+ font-size: 1.0rem;
+ font-weight: 500;
+ margin: 0;
+}
+
+.obj-returned-div {
+ padding-top: 10px;
+ padding-bottom: 10px;
+}
+
.run {
- color: blue;
+ color: #00ab66;
+}
+
+.explain {
+ color: blue;
+}
+
+.stop {
+ color: #cf142b;
}
.dataverses {
@@ -103,6 +148,10 @@
font-weight: 500 !important;
}
+.mat-action-row {
+ padding-right: 24px;
+}
+
.d1 {
margin-right: 15px;
}
@@ -124,4 +173,24 @@
.sql-history {
width: 500px !important;
-}
\ No newline at end of file
+}
+
+.plan-output-format {
+ width: 90px !important;
+}
+
+.content {
+ border-style: dotted;
+ border-width: 1px;
+ border-color: black;
+}
+
+.warning-msgs {
+ overflow: scroll;
+ white-space: initial;
+}
+
+.error-line {
+ background-color: red;
+ color: red;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
index 7702db7..582b2f2 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/input.component.ts
@@ -1,35 +1,43 @@
-/*
-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
+ /*
+ 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.
+ */
+ import {Component, EventEmitter, Output, ViewChild} from '@angular/core';
+ import { Observable } from 'rxjs';
+ import { Store } from '@ngrx/store';
+ import * as sqlQueryActions from '../../shared/actions/query.actions';
+ import * as sqlCancelActions from '../../shared/actions/cancel.actions';
+ import * as appActions from '../../shared/actions/app.actions'
+ import * as dataverseActions from '../../shared/actions/dataverse.actions'
+ import * as CodeMirror from 'codemirror';
+ import 'codemirror/addon/edit/closebrackets';
+ import 'codemirror/mode/sql/sql';
+ import 'codemirror/addon/hint/show-hint';
+ import 'codemirror/addon/hint/sql-hint';
- 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.
-*/
-import { Component, ViewChild } from '@angular/core';
-import { Observable } from 'rxjs';
-import { Store } from '@ngrx/store';
-import * as sqlQueryActions from '../../shared/actions/query.actions';
-import * as appActions from '../../shared/actions/app.actions'
-import * as dataverseActions from '../../shared/actions/dataverse.actions'
-import * as CodeMirror from 'codemirror';
-/*
- * Query component
- * has editor (codemirror)
- */
-@Component({
+ /*
+ * Query component
+ * has editor (codemirror)
+ */
+ @Component({
moduleId: module.id,
selector: 'awc-query',
templateUrl: 'input.component.html',
styleUrls: ['input.component.scss']
-})
+ })
-export class InputQueryComponent {
+ export class InputQueryComponent {
+ @Output() inputToOutputEmitter: EventEmitter<Object> = new EventEmitter<Object>();
+ @Output() isErrorEmitter: EventEmitter<Boolean> = new EventEmitter<Boolean>();
+ @Output() hideOutputEmitter: EventEmitter<Boolean> = new EventEmitter<Boolean>();
+
currentQuery = 0;
queryString: string = "";
metricsString: {};
@@ -42,6 +50,12 @@
queryMetrics: {};
querySuccess$: Observable <any> ;
querySuccess: Boolean = false;
+ querySuccessWarnings: Boolean = false;
+ queryWarnings: any;
+ queryWarnings$: Observable<any> ;
+ queryWarningsMessages: string[] = [];
+ queryWarningsCount = 0;
+ queryWarningsShow: Boolean = false;
queryError$: Observable <any> ;
queryError: Boolean = false;
queryErrorMessage$: Observable <any> ;
@@ -50,6 +64,8 @@
queryPrepared: {};
queryPlanFormats$: Observable <any> ;
queryPlanFormats: {};
+ cancelQuery$: Observable<any>;
+ isCanceled: boolean = false;
preparedQueryCount: number;
previousDisabled = true;
nextDisabled = true;
@@ -58,7 +74,7 @@
dataverses$: Observable<any>;
dataverses: any;
defaultDataverse = 'Default';
- selected = 'None';
+ selected = 'Default';
history = [];
currentHistory = 0;
viewCurrentHistory = 0; // for the view
@@ -67,283 +83,404 @@
none = 'None';
planFormat = 'JSON';
historyStringSelected = '';
+ historyIdxSelected = 0;
formatOptions = 'JSON';
+ outputOptions = 'JSON';
+ explainMode = false;
+ inputToOutput = {};
+ objectsReturned = 0;
/* Codemirror configuration */
- codemirrorConfig = {
- mode: "asterix",
- lineWrapping: true,
- showCursorWhenSelecting: true,
- autofocus: true,
- lineNumbers: true,
- };
+ //sql++ keywords
+ sqlppKeywords = "alter and as asc between by count create delete desc distinct drop from group having in insert into " +
+ "is join like not on or order select set union update values where limit use let dataverse dataset exists with index type" +
+ "inner outer offset value type if exists declare function";
+
+ //sql++ builtin types
+ sqlppTypes = "boolean tinyint smallint integer int bigint string float double binary point line rectangle circle polygon" +
+ "uuid object array multiset date time datetime duration year_month_duration day_time_duration interval"
constructor(private store: Store < any > ) {
- this.currentQuery = 0;
- this.querySuccess$ = this.store.select(s => s.sqlQuery.successHash);
- this.querySuccess$.subscribe((data: any) => {
- this.querySuccesResults = data;
- this.querySpinnerVisible = false;
- if (data != undefined && data[this.currentQuery] === true) {
- this.querySuccess = true;
- } else {
- this.querySuccess = false;
- }
- })
+ this.currentQuery = 0;
+ this.querySuccess$ = this.store.select(s => s.sqlQuery.successHash);
+ this.querySuccess$.subscribe((data: any) => {
+ this.isCanceled = false;
+ this.querySuccesResults = data;
+ this.querySpinnerVisible = false;
+ if (data != undefined && data[this.currentQuery] === true) {
+ this.querySuccess = true;
+ } else {
+ this.querySuccess = false;
+ }
+ })
- /* Watching for SQL Input Errors in current Query */
- this.queryError$ = this.store.select(s => s.sqlQuery.errorHash);
- this.queryError$.subscribe((data: any) => {
- this.querySpinnerVisible = false;
- if (data != undefined && data[this.currentQuery] === true) {
- this.queryError = true;
- this.showErrors();
- } else {
- this.queryError = false;
- }
- })
+ /* Watching for SQL Input Errors in current Query */
+ this.queryError$ = this.store.select(s => s.sqlQuery.errorHash);
+ this.queryError$.subscribe((data: any) => {
+ this.querySpinnerVisible = false;
+ if (data != undefined && data[this.currentQuery] === true) {
+ this.queryError = true;
+ this.showErrors();
+ } else {
+ this.queryError = false;
+ }
+ })
- /* Watching for Queries that are in prepared state,
- * those are SQL strings that still has not been executed
- */
- this.queryPrepared$ = this.store.select(s => s.sqlQuery.sqlQueryPrepared);
- this.queryPrepared$.subscribe((data: any) => {
- if (data) {
- this.queryPrepared = data
- this.preparedQueryCount = Object.keys(this.queryPrepared).length;
- if (this.preparedQueryCount == 0) {
- // Initialize Query Editor, prepare the default query
- this.queryPrepare = {
- editorId: String(this.currentQuery),
- queryString: this.queryString,
- planFormat: this.planFormat
- };
- this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
- } else {
- if (this.queryPrepared && this.queryPrepared[this.currentQuery]) {
- this.queryString = this.queryPrepared[this.currentQuery];
- }
- }
- } else {
- this.queryPrepared = {};
- }
- })
+ /* Watching for Queries that are in prepared state,
+ * those are SQL strings that still has not been executed
+ */
+ this.queryPrepared$ = this.store.select(s => s.sqlQuery.sqlQueryPrepared);
+ this.queryPrepared$.subscribe((data: any) => {
+ if (data) {
+ this.queryPrepared = data;
+ this.preparedQueryCount = Object.keys(this.queryPrepared).length;
- /* Watching for Metrics */
- this.queryMetrics$ = this.store.select(s => s.sqlQuery.sqlQueryMetrics);
- this.queryMetrics$.subscribe((data: any) => {
- if (data != undefined) {
- this.queryMetrics = Object.assign(data);
- if (this.queryMetrics && this.queryMetrics[this.currentQuery]) {
- this.metricsString = "SUCCESS: ";
- this.metricsString += " Execution time: " + this.queryMetrics[this.currentQuery].executionTime;
- this.metricsString += " Elapsed time: " + this.queryMetrics[this.currentQuery].elapsedTime;
- this.metricsString += " Size: " + (this.queryMetrics[this.currentQuery].resultSize/1024).toFixed(2) + ' Kb';
- }
- } else {
- this.queryMetrics = {};
- }
- })
+ if (this.queryPrepared && this.queryPrepared[this.currentQuery]) {
+ this.queryString = this.queryPrepared[this.currentQuery];
+ }
+ } else {
+ this.queryPrepared = {};
+ }
+ })
- /* Watching for SQL Input Errors: Error Message stored in Query Cache */
- this.queryErrorMessage$ = this.store.select(s => s.sqlQuery.sqlQueryErrorHash);
- this.queryErrorMessage$.subscribe((data: any) => {
- if (data) {
- this.queryErrorMessages = data;
- this.showErrors();
- } else {
- this.queryErrorMessages = {};
- }
- })
-
- /* Watching for SQL Input Errors: Error Message stored in Query Cache */
- this.queryPlanFormats$ = this.store.select(s => s.sqlQuery.sqlQueryPlanFormatHash);
- this.queryPlanFormats$.subscribe((data: any) => {
- if (data) {
- this.queryPlanFormats = data;
- } else {
- this.queryPlanFormats = {};
- }
- })
-
- this.preparedQueryCount = 0;
- // Initialize Query Editor, prepare the default query
- this.queryPrepare = {
- editorId: String(this.currentQuery),
- queryString: this.queryString,
- planFormat: this.formatOptions
- };
- this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
- // lets inform other views what's the current SQL editor
- this.store.dispatch(new appActions.setEditorIndex(String(this.currentQuery)));
- }
-
- ngOnInit() {
- this.dataverses$ = this.store.select(s => s.dataverse.dataverses.results);
- // Watching for Dataverses
- this.dataverses$ = this.store.select(s => s.dataverse.dataverses.results);
- this.dataverses$.subscribe((data: any[]) => {
- this.dataverses = data;
- this.defaultDataverse = ''
- });
- this.store.dispatch(new dataverseActions.SelectDataverses('-'), );
- }
-
- showMetrics() {
- this.querySuccess = false;
- if (this.queryMetrics && this.queryMetrics[this.currentQuery] && this.querySuccesResults[this.currentQuery]) {
+ /* Watching for Metrics */
+ this.queryMetrics$ = this.store.select(s => s.sqlQuery.sqlQueryMetrics);
+ this.queryMetrics$.subscribe((data: any) => {
+ if (data != undefined) {
+ this.queryMetrics = Object.assign(data);
+ if (this.queryMetrics && this.queryMetrics[this.currentQuery]) {
+ this.objectsReturned = this.queryMetrics[this.currentQuery].resultCount;
+ this.isErrorEmitter.emit(false);
+ this.hideOutputEmitter.emit(false);
this.metricsString = "SUCCESS: ";
+
+ if ('warningCount' in this.queryMetrics[this.currentQuery]) {
+ this.metricsString += " (WITH " + this.queryMetrics[this.currentQuery].warningCount + " WARNINGS)";
+ }
+
this.metricsString += " Execution time: " + this.queryMetrics[this.currentQuery].executionTime;
this.metricsString += " Elapsed time: " + this.queryMetrics[this.currentQuery].elapsedTime;
this.metricsString += " Size: " + (this.queryMetrics[this.currentQuery].resultSize/1024).toFixed(2) + ' Kb';
- this.querySuccess = true;
+ }
+ } else {
+ this.queryMetrics = {};
+ this.objectsReturned = 0;
}
+ })
+
+ /* Watching for SQL Input Errors: Error Message stored in Query Cache */
+ this.queryErrorMessage$ = this.store.select(s => s.sqlQuery.sqlQueryErrorHash);
+ this.queryErrorMessage$.subscribe((data: any) => {
+ if (data) {
+ this.queryErrorMessages = data;
+ this.showErrors();
+ } else {
+ this.queryErrorMessages = {};
+ }
+ })
+
+ /* Watching for SQL Input Warnings in current Query*/
+ this.queryWarnings$ = this.store.select(s => s.sqlQuery.sqlQueryWarnings);
+ this.queryWarnings$.subscribe((data: any) => {
+ if (data) {
+ this.queryWarnings = data;
+ this.showWarnings();
+ } else {
+ this.queryWarnings = {};
+ }
+ })
+
+ /*
+ * Watching for SQL Cancel Query
+ */
+ this.cancelQuery$ = this.store.select(s => s.cancelQuery.success);
+ this.cancelQuery$.subscribe((data: any) => {
+ if (data) {
+ this.querySpinnerVisible = false;
+ this.isCanceled = true;
+ }
+ })
+
+ /* Watching for SQL Input Errors: Error Message stored in Query Cache */
+ this.queryPlanFormats$ = this.store.select(s => s.sqlQuery.sqlQueryPlanFormatHash);
+ this.queryPlanFormats$.subscribe((data: any) => {
+ if (data) {
+ this.queryPlanFormats = data;
+ } else {
+ this.queryPlanFormats = {};
+ }
+ })
+
+ this.preparedQueryCount = 0;
+ // Initialize Query Editor, prepare the default query
+ this.saveQuery(String(this.currentQuery), this.queryString, this.formatOptions, "JSON");
+
+ // lets inform other views what's the current SQL editor
+ this.store.dispatch(new appActions.setEditorIndex(String(this.currentQuery)));
+ }
+
+ ngOnInit() {
+ this.dataverses$ = this.store.select(s => s.dataverse.dataverses.results);
+ // Watching for Dataverses
+ this.dataverses$ = this.store.select(s => s.dataverse.dataverses.results);
+ this.dataverses$.subscribe((data: any[]) => {
+ this.dataverses = data;
+ this.defaultDataverse = ''
+ });
+ this.store.dispatch(new dataverseActions.SelectDataverses('-'), );
+ }
+
+ showMetrics() {
+ this.querySuccess = false;
+ if (this.queryMetrics && this.queryMetrics[this.currentQuery] && this.querySuccesResults[this.currentQuery]) {
+ this.objectsReturned = this.queryMetrics[this.currentQuery].resultCount;
+ this.metricsString = "SUCCESS: ";
+
+ if ('warningCount' in this.queryMetrics[this.currentQuery]) {
+ this.metricsString += " [WITH " + this.queryMetrics[this.currentQuery].warningCount + " WARNING(S)]";
+ }
+
+ this.metricsString += " Execution time: " + this.queryMetrics[this.currentQuery].executionTime;
+ this.metricsString += " Elapsed time: " + this.queryMetrics[this.currentQuery].elapsedTime;
+ this.metricsString += " Size: " + (this.queryMetrics[this.currentQuery].resultSize/1024).toFixed(2) + ' Kb';
+
+ this.querySuccess = true;
+ }
+ }
+
+ showWarnings() {
+ this.queryWarningsShow = false;
+ if (this.queryWarnings && this.queryWarnings[this.currentQuery]) {
+ let warningObject = this.queryWarnings[this.currentQuery];
+
+ this.queryWarningsMessages = [];
+ this.queryWarningsCount = warningObject.length;
+ if (warningObject.length != 0) {
+ for (let warning of warningObject.reverse()) {
+ let warningString = "WARNING: Code: " + JSON.stringify(warning.code, null, 8);
+ warningString += " " + JSON.stringify(warning.msg, null, 8);
+
+ this.queryWarningsMessages.push(warningString);
+ }
+
+ this.queryWarningsShow = true;
+ }
+ }
}
showErrors() {
- this.queryError = false;
- if (this.queryErrorMessages && this.queryErrorMessages[this.currentQuery]) {
- let errorObject = this.queryErrorMessages[this.currentQuery];
- if (errorObject.length != 0) {
- this.queryErrorMessageString = "ERROR: Code: " + JSON.stringify(errorObject[0].code, null, 8);
- this.queryErrorMessageString += " " + JSON.stringify(errorObject[0].msg, null, 8);
- this.queryError = true;
- }
+ this.queryError = false;
+ if (this.queryErrorMessages && this.queryErrorMessages[this.currentQuery]) {
+ let errorObject = this.queryErrorMessages[this.currentQuery];
+ if (errorObject.length != 0) {
+ this.queryErrorMessageString = "ERROR: Code: " + JSON.stringify(errorObject[0].code, null, 8);
+ this.queryErrorMessageString += " " + JSON.stringify(errorObject[0].msg, null, 8);
+ this.queryError = true;
+
+ this.isErrorEmitter.emit(true);
}
+ }
}
- getQueryResults(queryString: string, planFormat: string) {
- let QueryOrder = this.currentQuery;
- this.queryRequest = {
- requestId: String(QueryOrder),
- queryString: queryString,
- planFormat: planFormat
- };
- this.store.dispatch(new sqlQueryActions.ExecuteQuery(this.queryRequest));
- this.querySpinnerVisible = true;
+ getQueryResults(queryString: string, planFormat: string, outputFormat: string) {
+ let QueryOrder = this.currentQuery;
+ this.queryRequest = {
+ requestId: String(QueryOrder),
+ queryString: queryString,
+ planFormat: planFormat,
+ format: outputFormat
+ };
+ this.store.dispatch(new sqlQueryActions.ExecuteQuery(this.queryRequest));
+ this.querySpinnerVisible = true;
+ }
+
+ onClickExplain() {
+ let use_regex = /use .*?;/i
+
+ let explainString = "";
+
+ if (use_regex.test(this.queryString))
+ explainString = this.queryString.replace(use_regex, "$& explain ");
+ else
+ explainString = "explain " + this.queryString;
+
+ this.runQuery(explainString, true);
+
+ this.explainMode = true;
+ this.sendInputToOutput();
}
onClickRun() {
- let planFormat = this.formatOptions;
- this.getQueryResults(this.queryString, planFormat); // .replace(/\n/g, " "));
- if (this.history.length === 0) {
- this.history.push('Clear');
- }
- this.history.push(this.queryString);
- this.currentHistory = this.history.length - 1;
- this.viewCurrentHistory = this.history.length;
+ this.explainMode = false;
+ this.sendInputToOutput();
+
+ this.runQuery(this.queryString, this.explainMode);
+ }
+
+ runQuery(stringToRun: string, isExplain: boolean) {
+ this.autodetectDataverse();
+
+ if (this.queryString != this.queryPrepared[Object.keys(this.queryPrepared).length - 1]) {
+ //not the same query as before, currentQuery needs to be incremented and another needs to be dispatched
+ //increment currentQuery
+ if (this.queryPrepared[Object.keys(this.queryPrepared).length - 1] != '')
+ this.currentQuery = Object.keys(this.queryPrepared).length;
+
+ this.saveQuery(String(this.currentQuery), this.queryString, this.formatOptions, this.outputOptions);
+
+ this.history.unshift({query: this.queryString, index: this.currentQuery});
+
+ //this.currentHistory = this.history.length - 1;
+ //this.viewCurrentHistory = this.history.length;
+
+ //display
+ let currentQueryIndex = String(this.currentQuery);
+ this.store.dispatch(new appActions.setEditorIndex(currentQueryIndex));
+ this.editor.focus();
+
+ } else {
+ //the same query as before, currentQuery is not incremented
+ //save the current Query
+ this.saveQuery(String(this.currentQuery), this.queryString, this.formatOptions, this.outputOptions);
+ }
+
+ let planFormat = this.formatOptions;
+ let outputFormat = this.outputOptions;
+ this.historyIdxSelected = this.currentQuery;
+
+ this.getQueryResults(stringToRun, planFormat, outputFormat); // .replace(/\n/g, " "));
+ }
+
+ onClickStop() {
+ let cancel_request = {
+ requestId: String(this.currentQuery)
+ }
+
+ this.store.dispatch(new sqlCancelActions.CancelQuery(cancel_request));
}
onClickNew() {
- // Saving first
- this.queryPrepare = {
- editorId: String(this.currentQuery),
- queryString: this.queryString,
- planFormat: this.formatOptions
- };
- this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
- // Prepare a new Query String, cleanup screen messages
- this.currentQuery = Object.keys(this.queryPrepared).length;
- this.queryString = "";
- this.editor.getDoc().setValue(this.queryString);
- this.queryErrorMessageString = "";
- this.metricsString = "";
- this.querySuccess = false;
- this.queryError = false;
- this.queryPrepare = {
- editorId: String(this.currentQuery),
- queryString: "",
- planFormat: this.formatOptions
- };
- this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
- // lets inform other views what's the current SQL editor
- let currentQueryIndex = String(this.currentQuery);
- this.store.dispatch(new appActions.setEditorIndex(currentQueryIndex));
- this.selected = "None";
- this.editor.focus();
+ // Saving first
+ this.saveQuery(String(this.currentQuery), this.queryString, this.formatOptions, this.outputOptions);
+
+ //let output section know to hide
+ this.hideOutputEmitter.emit(true);
+
+ this.historyIdxSelected = -1;
+
+ // Prepare a new Query String, cleanup screen messages
+ this.currentQuery = Object.keys(this.queryPrepared).length;
+ this.queryString = "";
+ this.editor.getDoc().setValue(this.queryString);
+ this.queryErrorMessageString = "";
+ this.metricsString = "";
+ this.querySuccess = false;
+ this.queryError = false;
+ this.queryWarningsShow = false;
+
+ this.saveQuery(String(this.currentQuery), "", this.formatOptions, this.outputOptions);
+ // lets inform other views what's the current SQL editor
+ let currentQueryIndex = String(this.currentQuery);
+ this.store.dispatch(new appActions.setEditorIndex(currentQueryIndex));
+ this.selected = "Default";
+ this.editor.focus();
}
onClickClear() {
- let queryClear = {
- editorId: String(this.currentQuery),
- queryString: "",
- planFormat: "JSON"
- };
- this.store.dispatch(new sqlQueryActions.CleanQuery(queryClear));
- this.queryErrorMessageString = "";
- this.queryString = "";
- this.metricsString = "";
- this.dataverseSelected();
- this.editor.getDoc().setValue(this.queryString);
- this.editor.focus();
+ let queryClear = {
+ editorId: String(this.currentQuery),
+ queryString: "",
+ planFormat: "JSON"
+ };
+ this.store.dispatch(new sqlQueryActions.CleanQuery(queryClear));
+ this.queryErrorMessageString = "";
+ this.queryString = "";
+ this.metricsString = "";
+ this.queryWarningsCount = 0;
+ this.queryWarningsMessages = [];
+
+ this.dataverseSelected();
+ this.editor.getDoc().setValue(this.queryString);
+ this.editor.execCommand('goDocEnd')
+ this.editor.focus();
+
+ //hide output on clear
+ this.hideOutputEmitter.emit(true);
}
onClickPrevious() {
- if (this.currentQuery > 0) {
- this.nextSQLEditor(-1);
- }
+ if (this.currentQuery > 0) {
+ this.nextSQLEditor(this.currentQuery - 1);
+ }
}
onClickNext() {
- if (this.currentQuery < this.preparedQueryCount - 1) {
- this.nextSQLEditor(1);
+ if (this.currentQuery < this.preparedQueryCount - 1) {
+ this.nextSQLEditor(this.currentQuery + 1);
+ }
+ else {
+ if (this.queryString != '') {
+ this.onClickNew();
}
+ }
}
checkNext() {
- if (this.currentQuery == this.preparedQueryCount - 1) {
- return true;
- } else {
- return false;
- }
+ if (this.currentQuery == this.preparedQueryCount) {
+ return true;
+ } else {
+ return false;
+ }
}
checkPrevious() {
- if (this.currentQuery == 0) {
- return true;
- } else {
- return false;
- }
+ if (this.currentQuery == 0) {
+ return true;
+ } else {
+ return false;
+ }
}
nextSQLEditor(next) {
- // Saving First
- this.queryPrepare = {
- editorId: String(this.currentQuery),
- queryString: this.queryString,
- planFormat: this.formatOptions
- };
- this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
- this.currentQuery = this.currentQuery + next;
- this.queryErrorMessageString = "";
- this.metricsString = "";
+ // Saving First
+ this.saveQuery(String(this.currentQuery), this.queryString, this.formatOptions, this.outputOptions);
- // Retrieve Metrics or Error Message if Query was executed
- this.showMetrics();
+ //this.currentQuery = this.currentQuery + next;
+ this.currentQuery = next;
+ this.queryErrorMessageString = "";
+ this.metricsString = "";
- // Retrieve Metrics or Error Message if Query was executed
- this.showErrors();
+ // Retrieve Metrics or Error Message if Query was executed
+ this.showMetrics();
- // Retrieve the prepared SQL string
- this.queryString = this.queryPrepared[this.currentQuery];
- this.editor.getDoc().setValue(this.queryString);
+ // Retrieve Metrics or Error Message if Query was executed
+ this.showErrors();
- // Retrieve the prepared SQL plan Format
- this.formatOptions = this.queryPlanFormats[this.currentQuery];
+ // Retrieve the prepared SQL string
+ this.queryString = this.queryPrepared[this.currentQuery];
+ this.editor.getDoc().setValue(this.queryString);
- // lets inform other views what's the current SQL editor
- let currentQueryIndex = String(this.currentQuery);
+ // Select the query from the QUERY History
+ this.historyIdxSelected = this.currentQuery;
- // Inform the app we are now active in next editor
- this.store.dispatch(new appActions.setEditorIndex(currentQueryIndex));
+ this.autodetectDataverse();
+
+ // Retrieve the prepared SQL plan Format
+ this.formatOptions = this.queryPlanFormats[this.currentQuery];
+
+ // lets inform other views what's the current SQL editor
+ let currentQueryIndex = String(this.currentQuery);
+
+ // Inform the app we are now active in next editor
+ this.store.dispatch(new appActions.setEditorIndex(currentQueryIndex));
}
onClickInputCardCollapse() {
- this.collapse = !this.collapse;
- if (this.collapse) {
- this.input_expanded_icon = 'expand_more';
- } else {
- this.input_expanded_icon = 'expand_less';
- }
+ this.collapse = !this.collapse;
+ if (this.collapse) {
+ this.input_expanded_icon = 'expand_more';
+ } else {
+ this.input_expanded_icon = 'expand_less';
+ }
}
/**
@@ -351,65 +488,141 @@
*/
@ViewChild('editor') editor: CodeMirror.Editor;
ngAfterViewInit() {
- this.codemirrorInit(this.codemirrorConfig);
+ this.codemirrorInit();
}
/**
* Initialize codemirror
*/
- codemirrorInit(config) {
- this.editor = CodeMirror.fromTextArea(this.editor.nativeElement, config);
- this.editor.setSize(null, 'auto');
- this.editor.getDoc().setValue(this.queryString);
- this.editor.on('changes', () => {
- this.queryString = this.editor.getValue();
- });
+ codemirrorInit() {
+ this.editor = CodeMirror.fromTextArea(this.editor.nativeElement, {
+ mode: {
+ name: "sql",
+ keywords: this.set(this.sqlppKeywords),
+ builtin: this.set(this.sqlppTypes),
+ atoms: this.set("true false null missing"),
+ dateSQL: this.set("date time datetime duration year_month_duration day_time_duration interval"),
+ support: this.set("ODBCdotTable doubleQuote binaryNumber hexNumber commentSlashSlash")
+ },
+ lineWrapping: true,
+ showCursorWhenSelecting: true,
+ autofocus: true,
+ lineNumbers: true,
+ autoCloseBrackets: {
+ pairs: "()[]{}''\"\"``",
+ closeBefore: ")]}'\":;>",
+ triples: "",
+ explode: "[]{}()",
+ override: true
+ },
+ extraKeys: {"Ctrl-Space": "autocomplete"},
+ hint: CodeMirror.hint.sql,
+ });
+ this.editor.setSize(null, 'auto');
+ this.editor.getDoc().setValue(this.queryString);
+ this.editor.on('changes', () => {
+ this.queryString = this.editor.getValue();
+ });
}
dataverseSelected() {
- if (this.selected == undefined) {
- this.queryString = 'None';
- } else if (this.selected === 'None') {
- this.queryString = '';
- this.selected = 'None';
- } else {
- this.queryString = 'Use ' + this.selected + '; ';
- }
- this.editor.getDoc().setValue(this.queryString);
- this.editor.focus();
+ if (this.selected == undefined) {
+ this.queryString = 'None';
+ } else if (this.selected === 'None' || this.selected === 'Default') {
+ this.queryString = '';
+ this.selected = 'Default';
+ } else {
+ this.queryString = 'USE ' + this.selected + '; \n';
+ }
+ this.editor.getDoc().setValue(this.queryString);
+ this.editor.execCommand('goDocEnd')
+ this.editor.focus();
}
- historySelected() {
- if (this.historyStringSelected == undefined) {
- this.historyStringSelected = '';
- } else if (this.historyStringSelected === 'Clear') {
- this.history = [];
- this.historyStringSelected = '';
- }
- this.queryString = this.historyStringSelected;
- this.editor.getDoc().setValue(this.queryString);
- this.editor.focus();
+ historySelected(idx: number) {
+ if (this.historyStringSelected == undefined) {
+ this.historyStringSelected = '';
+ } else if (this.historyStringSelected === 'Clear') {
+ this.history = [];
+ this.historyStringSelected = '';
+ }
+
+ this.nextSQLEditor(idx);
}
planFormatSelected() {}
onClickNextHistory() {
- if (this.currentHistory < this.history.length - 1) {
- this.currentHistory++;
- this.viewCurrentHistory++;
- this.queryString = this.history[this.currentHistory];
- this.editor.getDoc().setValue(this.queryString);
- this.editor.focus();
- }
+ if (this.currentHistory < this.history.length - 1) {
+ this.currentHistory++;
+ this.viewCurrentHistory++;
+ this.queryString = this.history[this.currentHistory];
+ this.editor.getDoc().setValue(this.queryString);
+ this.editor.focus();
+ }
}
onClickPrevHistory() {
- if (this.currentHistory > 0) {
- this.currentHistory--;
- this.viewCurrentHistory--;
- this.queryString = this.history[this.currentHistory];
- this.editor.getDoc().setValue(this.queryString);
- this.editor.focus();
- }
+ if (this.currentHistory > 0) {
+ this.currentHistory--;
+ this.viewCurrentHistory--;
+ this.queryString = this.history[this.currentHistory];
+ this.editor.getDoc().setValue(this.queryString);
+ this.editor.focus();
+ }
}
-}
\ No newline at end of file
+
+ saveQuery(editorId, queryString, planFormat, outputFormat) {
+ this.queryPrepare = {
+ editorId: String(editorId),
+ queryString: queryString,
+ planFormat: planFormat,
+ format: outputFormat
+ };
+ this.store.dispatch(new sqlQueryActions.PrepareQuery(this.queryPrepare));
+ }
+
+ sendInputToOutput() {
+ this.inputToOutput['isExplain'] = this.explainMode;
+ this.inputToOutput['outputFormat'] = this.outputOptions;
+
+ this.inputToOutputEmitter.emit(this.inputToOutput);
+ }
+
+ autodetectDataverse() {
+ let dataverseRegex = /use (.*?);/i
+
+ let matches = this.queryString.match(dataverseRegex);
+
+ let detectedDataverse = "";
+
+ if (matches) {
+ if (matches.length == 2) {
+ detectedDataverse = matches[1];
+ }
+ }
+
+ if (detectedDataverse != "") {
+ let dataverseNames = this.dataverses.map(function (e) {
+ return e.DataverseName
+ });
+
+ if (dataverseNames.includes(detectedDataverse)) {
+ this.selected = detectedDataverse;
+ } else {
+ this.selected = "Default";
+ }
+ } else {
+ this.selected = "Default";
+ }
+ }
+
+ set(str: string) {
+ let obj = {};
+ let words = str.split(" ");
+ for (let i = 0; i < words.length; i++) {
+ obj[words[i]] = true;
+ }
+ return obj;
+ }
+ }
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.html
index e8869ca..6810a413 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.html
@@ -11,12 +11,109 @@
See the License for the specific language governing permissions and
limitations under the License.
*/-->
-<div class="inspector-dialog">
- <p mat-dialog-title class="header">METADATA INSPECTOR</p>
- <mat-dialog-content>
- <pre class="content">{{data}}</pre>
+<div class="inspector-dialog" cdkDrag cdkDragRootElement=".cdk-overlay-pane">
+ <div class="top-row" cdkDragHandle>
+ <p mat-dialog-title class="header" *ngIf="data.MetadataType === 'dataset'">DATASET: {{data.DatasetName}}</p>
+ <p mat-dialog-title class="header" *ngIf="data.MetadataType === 'datatype'">DATATYPE: {{data.DatatypeName}}</p>
+ <p mat-dialog-title class="header" *ngIf="data.MetadataType === 'index'">INDEX: {{data.IndexName}}</p>
+ <p mat-dialog-title class="header" *ngIf="data.MetadataType === 'function'">FUNCTION: {{data.Name}}</p>
+ <button mat-icon-button class='input-button' (click)="onClickClose()" matTooltip="Close Inspector Window"><mat-icon>close</mat-icon></button>
+ </div>
+ <mat-dialog-content class="metadata-inspector" *ngIf="showGuts == false">
+ <div *ngIf="data.MetadataType === 'dataset'">
+ <pre class="content"><b>Dataverse: </b>{{data.DataverseName}}</pre>
+ <pre class="content"><b>Dataset: </b>{{data.DatasetName}}</pre>
+ <pre class="content"><b>Datatype Name: </b>{{data.DatatypeName}}</pre>
+ <div *ngIf="data.InternalDetails.PrimaryKey.length > 0">
+ <pre class="list-title content"><b>Primary Keys:</b></pre>
+ <li class="content" *ngFor="let pkey of data.InternalDetails.PrimaryKey">{{pkey}}</li>
+ </div>
+ <pre class="content"><b>Sample:</b></pre>
+ <mat-card class="sample-card">
+ <mat-card-content>
+ <pre class="content" *ngIf="data.sample != undefined">{{data.sample}}</pre>
+ <pre class="content" *ngIf="data.sample === undefined">No Data inputed</pre>
+ </mat-card-content>
+ </mat-card>
+ </div>
+ <div *ngIf="data.MetadataType === 'datatype'">
+ <pre class="content"><b>Dataverse: </b>{{data.DataverseName}}</pre>
+ <pre class="content"><b>Datatype Name: </b>{{data.DatatypeName}}</pre>
+ <pre class="list-title content list-tag" *ngIf="data.Derived.Tag === 'ORDEREDLIST'"><b>[</b></pre>
+ <pre class="list-title content list-tag" *ngIf="data.Derived.Tag === 'UNORDEREDLIST'"><b>{{</b></pre>
+ <pre class="list-title content" *ngIf="data.Derived.Tag === 'RECORD'"><b>Fields:</b></pre>
+ <mat-tree [dataSource]="dataSource" [treeControl]="treeControl">
+ <mat-tree-node *matTreeNodeDef="let node;" matTreeNodeToggle>
+ <mat-expansion-panel hideToggle disabled class="tree-node mat-elevation-z0 right-aligned-header" (click)="$event.stopPropagation();">
+ <mat-expansion-panel-header [collapsedHeight]="'25px'">
+ <mat-panel-title class="not-nested-title" *ngIf="node.DatatypeType !== undefined">{{node.DatatypeName}} ({{node.DatatypeType}})</mat-panel-title>
+ <mat-panel-title class="not-nested-title" *ngIf="node.DatatypeType === undefined">{{node.DatatypeName}}</mat-panel-title>
+
+ <mat-panel-description class="not-nested-options" *ngIf="node.isNullable == true && node.isMissable == false">Nullable | Required</mat-panel-description>
+ <mat-panel-description class="not-nested-options" *ngIf="node.isNullable == true && node.isMissable == true">Nullable | Not Required</mat-panel-description>
+ <mat-panel-description class="not-nested-options" *ngIf="node.isNullable == false && node.isMissable == false">Not Nullable | Required</mat-panel-description>
+ <mat-panel-description class="not-nested-options" *ngIf="node.isNullable == false && node.isMissable == true">Not Nullable | Not Required</mat-panel-description>
+ </mat-expansion-panel-header>
+ </mat-expansion-panel>
+ </mat-tree-node>
+ <mat-nested-tree-node *matTreeNodeDef="let node;when: hasChild">
+ <div class="mat-tree-node">
+ <mat-expansion-panel class="tree-node mat-elevation-z2 right-aligned-header">
+ <mat-expansion-panel-header [collapsedHeight]="'25px'" [expandedHeight]="'35px'">
+ <mat-panel-title *ngIf="node.anonymous == false">{{node.DatatypeName}} ({{node.DatatypeType}})</mat-panel-title>
+ <mat-panel-title *ngIf="node.anonymous == true">{{node.DatatypeName}} (<i>{{node.DatatypeType}}</i>)</mat-panel-title>
+
+ <mat-panel-description *ngIf="node.isNullable == true && node.isMissable == false">Nullable | Required</mat-panel-description>
+ <mat-panel-description *ngIf="node.isNullable == true && node.isMissable == true">Nullable | Not Required</mat-panel-description>
+ <mat-panel-description *ngIf="node.isNullable == false && node.isMissable == false">Not Nullable | Required</mat-panel-description>
+ <mat-panel-description *ngIf="node.isNullable == false && node.isMissable == true">Not Nullable | Not Required</mat-panel-description>
+ </mat-expansion-panel-header>
+ <pre class="list-title content list-tag" *ngIf="node.OrderedList == true"><b>[</b></pre>
+ <pre class="list-title content list-tag" *ngIf="node.UnorderedList == true"><b>{{</b></pre>
+ <div [class.example-tree-invisible]="!treeControl.isExpanded(node)">
+ <ng-container matTreeNodeOutlet></ng-container>
+ </div>
+ <pre class="list-title content list-tag" *ngIf="node.OrderedList == true"><b>]</b></pre>
+ <pre class="list-title content list-tag" *ngIf="node.UnorderedList == true"><b>}}</b></pre>
+ </mat-expansion-panel>
+ </div>
+ </mat-nested-tree-node>
+ </mat-tree>
+ <pre class="list-title content list-tag" *ngIf="data.Derived.Tag === 'ORDEREDLIST'"><b>]</b></pre>
+ <pre class="list-title content list-tag" *ngIf="data.Derived.Tag === 'UNORDEREDLIST'"><b>}}</b></pre>
+ <p class="anon-note">Note: italicized = anonymous type</p>
+ </div>
+ <div *ngIf="data.MetadataType === 'index'">
+ <pre class="content"><b>Dataverse: </b>{{data.DataverseName}}</pre>
+ <pre class="content"><b>Index Name: </b>{{data.IndexName}}</pre>
+
+ <pre class="content" *ngIf="data.IsPrimary == true"><b>Index Type: </b>Primary</pre>
+ <pre class="content" *ngIf="data.IsPrimary == false"><b>Index Type: </b>Not Primary</pre>
+
+ <div *ngIf="data.SearchKey.length > 0">
+ <pre class="list-title content"><b>Search Key(s):</b></pre>
+ <li class="content" *ngFor="let skey of data.SearchKey">{{skey}}</li>
+ </div>
+ </div>
+ <div *ngIf="data.MetadataType === 'function'">
+ <pre class="content"><b>Dataverse: </b>{{data.DataverseName}}</pre>
+ <pre class="content"><b>Function Name: </b>{{data.Name}}</pre>
+ <pre class="content"><b>Arity: </b>{{data.Arity}}</pre>
+ <div *ngIf="data.Params.length > 0">
+ <pre class="content"><b>Parameters: </b></pre>
+ <li class="content" *ngFor="let param of data.Params">{{param}}</li>
+ </div>
+ <pre class="content"><b>Defintion: </b></pre>
+ <pre class="content">{{data.Definition}}</pre>
+
+ </div>
</mat-dialog-content>
+ <mat-dialog-content *ngIf="showGuts">
+ <pre class="content">{{data.guts}}</pre>
+ </mat-dialog-content>
+ <div class="spacer"></div>
<mat-action-row>
- <button mat-button class='input-button' (click)="onClickClose()">CLOSE</button>
+ <button mat-button class="input-button" (click)="onClickParsed()" *ngIf="hideJSONButton">SUMMARY</button>
+ <button mat-button class="input-button" (click)="onClickJSON()" *ngIf="hideJSONButton == false">JSON</button>
</mat-action-row>
-</div>
\ No newline at end of file
+</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.scss
index 0ca0461..72afdf9 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata-inspector.component.scss
@@ -12,19 +12,91 @@
limitations under the License.
*/
+::ng-deep.mat-dialog-container {
+ resize: both !important;
+ overflow: auto;
+}
+
.inspector-dialog {
font-size: 0.80rem;
font-weight: 500;
}
.header {
- font-size: 1.0rem;
+ font-size: 1.5rem;
font-weight: 500;
color: blue;
- border-bottom: 1px solid rgb(145, 152, 158);
}
.content {
margin-left: auto !important;
margin-right: auto !important;
-}
\ No newline at end of file
+}
+
+.list-title {
+ margin-bottom: 1px;
+}
+
+.metadata-inspector {
+ font-size: 1.0rem;
+}
+
+.metadata-inspector b {
+ color: blue !important;
+}
+
+.spacer {
+ margin-bottom: 30px;
+}
+
+.mat-expansion-panel {
+ margin-bottom: 1px;
+}
+
+.tree-node {
+ width: 100%;
+}
+
+.not-nested-title {
+ color: black !important;
+}
+
+.not-nested-options {
+ color: rgba(0, 0, 0, 0.54) !important;
+}
+
+.mat-tree-node {
+ min-height: 25px;
+}
+
+.mat-expansion-panel-header-description {
+ display: flex;
+ justify-content: flex-end;
+ flex: 0 0 auto;
+ padding-right: 5px;
+}
+
+.anon-note {
+ font-size: 15px;
+ color: #9e9e9e;
+}
+
+.sample-card {
+ max-height: 250px;
+ overflow:auto;
+ padding-bottom: 15px;
+}
+
+.list-tag {
+ margin-top: 0px !important;
+}
+
+.top-row {
+ display: flex;
+ flex-flow: row;
+ justify-content: space-between;
+ align-items: baseline;
+ border-bottom: 1px solid rgb(145, 152, 158);
+ cursor: move;
+ text-align: center;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.html
index 386520b..b50027e 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.html
@@ -13,51 +13,64 @@
*/ -->
<div class="wrapper">
<mat-expansion-panel hideToggle [expanded]="true">
- <mat-expansion-panel-header class="header-dataverse">
- <mat-icon>developer_board</mat-icon>
- <mat-panel-title>DATAVERSES</mat-panel-title>
- <mat-panel-description></mat-panel-description>
- </mat-expansion-panel-header>
- <section class="section">
- <li *ngFor="let dataverse of dataverses; index as i">
- <mat-checkbox [labelPosition]="before" class="margin" (change)="generateFilter(dataverse.DataverseName, $event, value, i)" [(ngModel)]="dataverse.active">{{dataverse.DataverseName}}</mat-checkbox>
- </li>
- <div class="refresh">
- <button mat-button class="refresh-button" (click)="refreshMetadata()" matTooltip="Click to refresh changes in AsterixDB"><mat-icon class="list-icon">cached</mat-icon>REFRESH</button>
- </div>
- </section>
- </mat-expansion-panel>
- <mat-expansion-panel hideToggle [expanded]="panelOpenState">
- <mat-expansion-panel-header #datasetsPanel class="header">
- <mat-icon>developer_board</mat-icon>
+ <mat-expansion-panel-header class="metadata-inspector-header">
+ <mat-panel-title>METADATA INSPECTOR</mat-panel-title>
+ </mat-expansion-panel-header>
+ <section>
+ <mat-expansion-panel hideToggle [expanded]="true">
+ <mat-expansion-panel-header class="header-dataverse">
+ <mat-panel-title class="title">DATAVERSES</mat-panel-title>
+ </mat-expansion-panel-header>
+ <section class="section">
+ <li *ngFor="let dataverse of dataverses; index as i">
+ <mat-checkbox [labelPosition]="'after'" class="margin" (change)="generateFilter(dataverse.DataverseName, $event, i)" [(ngModel)]="dataverse.active">{{dataverse.DataverseName}}</mat-checkbox>
+ </li>
+ <div class="refresh">
+ <button mat-button class="refresh-button" (click)="refreshMetadata()" matTooltip="Click to refresh changes in AsterixDB"><mat-icon class="list-icon">cached</mat-icon>REFRESH</button>
+ </div>
+ </section>
+ </mat-expansion-panel>
+ <mat-expansion-panel hideToggle [expanded]="panelOpenState">
+ <mat-expansion-panel-header #datasetsPanel class="header">
<mat-panel-title class="title">DATASETS</mat-panel-title>
<mat-panel-description></mat-panel-description>
- </mat-expansion-panel-header>
- <section class="section">
- <li *ngFor="let dataset of datasetsFiltered" (click)="openMetadataInspectorDialog(dataset)">
- {{dataset.DatasetName}}</li>
- </section>
- </mat-expansion-panel>
- <mat-expansion-panel hideToggle [expanded]="panelOpenState">
- <mat-expansion-panel-header class="header">
- <mat-icon>developer_board</mat-icon>
+ </mat-expansion-panel-header>
+ <section class="section">
+ <li *ngFor="let dataset of datasetsFiltered" (click)="openMetadataInspectorDialog(dataset, 'dataset')">
+ {{dataset.DatasetName}}</li>
+ </section>
+ </mat-expansion-panel>
+ <mat-expansion-panel hideToggle [expanded]="panelOpenState">
+ <mat-expansion-panel-header class="header">
<mat-panel-title class="title">DATATYPES</mat-panel-title>
<mat-panel-description></mat-panel-description>
- </mat-expansion-panel-header>
- <section class="section">
- <li *ngFor="let datatype of datatypesFiltered" (click)="openMetadataInspectorDialog(datatype)">
- {{datatype.DatatypeName}}</li>
- </section>
- </mat-expansion-panel>
- <mat-expansion-panel hideToggle [expanded]="panelOpenState">
- <mat-expansion-panel-header class="header">
- <mat-icon>developer_board</mat-icon>
+ </mat-expansion-panel-header>
+ <section class="section">
+ <li *ngFor="let datatype of datatypesFiltered" (click)="openMetadataInspectorDialog(datatype, 'datatype')">
+ {{datatype.DatatypeName}}</li>
+ </section>
+ </mat-expansion-panel>
+ <mat-expansion-panel hideToggle [expanded]="panelOpenState">
+ <mat-expansion-panel-header class="header">
<mat-panel-title class="title">INDEX</mat-panel-title>
<mat-panel-description></mat-panel-description>
- </mat-expansion-panel-header>
- <section class="section">
- <li *ngFor="let index of indexesFiltered" (click)="openMetadataInspectorDialog(index)">
- {{index.IndexName}}</li>
- </section>
+ </mat-expansion-panel-header>
+ <section class="section">
+ <li *ngFor="let index of indexesFiltered" (click)="openMetadataInspectorDialog(index, 'index')">
+ {{index.IndexName}}</li>
+ </section>
+ </mat-expansion-panel>
+ <mat-expansion-panel hideToggle [expanded]="panelOpenState">
+ <mat-expansion-panel-header class="header">
+ <mat-panel-title class="title">USER DEFINED FUNCTIONS</mat-panel-title>
+ <mat-panel-description></mat-panel-description>
+ </mat-expansion-panel-header>
+ <section class="section">
+ <li *ngFor="let function of functionsFiltered" (click)="openMetadataInspectorDialog(function, 'function')">
+ {{function.Name}}
+ </li>
+ </section>
+ </mat-expansion-panel>
+ </section>
</mat-expansion-panel>
-</div>
\ No newline at end of file
+</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.scss
index b79d774..3728cdb 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.scss
@@ -42,8 +42,7 @@
padding-right: 10px;
padding-bottom: 5px;
padding-top: 5px;
- //background-color: gainsboro;
- background-color: black;
+ background-color: gainsboro;
margin-bottom: 50px;
mat-expansion-panel {
border: none !important;
@@ -70,6 +69,15 @@
color: blue;
}
+.metadata-inspector-header {
+ font-size: 0.80rem;
+ font-weight: 500;
+ max-height: 42px;
+ min-height: 42px;
+ border-bottom: 1px solid gray;
+ margin-bottom: 5px;
+}
+
.title {
color:blue;
}
@@ -87,4 +95,4 @@
display: flex;
flex-flow: row;
align-items: center;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.ts
index 99c470c..835eeb8 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/metadata.component.ts
@@ -18,8 +18,41 @@
import * as datasetActions from '../../shared/actions/dataset.actions';
import * as datatypesActions from '../../shared/actions/datatype.actions';
import * as indexesActions from '../../shared/actions/index.actions';
+import * as functionActions from '../../shared/actions/function.actions';
import { ViewChild} from '@angular/core';
-import { MatDialog, MatDialogRef, MAT_DIALOG_DATA } from '@angular/material';
+import { MatDialog, MatDialogRef, MAT_DIALOG_DATA } from '@angular/material/dialog';
+import {MatTreeNestedDataSource} from "@angular/material/tree";
+import {NestedTreeControl} from "@angular/cdk/tree";
+import {Data} from "@angular/router";
+
+interface DataTypeNode {
+ DatatypeName: string;
+ DatatypeType?: string;
+ fields?: DataTypeNode[];
+ primitive: boolean;
+ isNullable?: boolean;
+ isMissable?: boolean;
+ OrderedList?: boolean;
+ UnorderedList?: boolean;
+ anonymous: boolean;
+}
+
+enum MetadataTypes {
+ record = "RECORD",
+ orderedList = "ORDEREDLIST",
+ unorderedList = "UNORDEREDLIST"
+}
+
+enum MetadataTypesNames {
+ record = "Record",
+ orderedList = "Ordered List",
+ unorderedList = "Unordered List"
+}
+
+enum MetadataListTypes {
+ orderedList = "OrderedList",
+ unorderedList = "UnorderedList"
+}
@Component({
moduleId: module.id,
@@ -40,9 +73,28 @@
indexesFiltered: any;
indexes$: Observable<any>;
indexes: any;
+ functionsFiltered: any;
+ functions$: Observable<any>;
+ functions: any;
+ curr_dialogRef: any;
+ dialogActive: boolean;
+
+ //added variables for sample
+ sampleDataset$: Observable<any>;
+ sampleDataset: any;
+
+ //variables for counting
+ countDataset$:Observable<any>;
+ countDataset: number;
+
+ //added variables for flattening
+ datatypesDict: Object;
+
+ dialogSamples = {};
constructor(private store: Store<any>, public dialog: MatDialog) {
this.refreshMetadata();
+ this.dialogActive = false;
}
ngOnInit() {
@@ -66,6 +118,7 @@
this.datatypes$.subscribe((data: any[]) => {
this.datatypes = data;
this.datatypesFiltered = this.filter(this.datatypes);
+ this.datatypesDict = this.createDatatypeDict(this.datatypes);
});
// Watching for indexes
@@ -74,6 +127,16 @@
this.indexes = data;
this.indexesFiltered = this.filter(this.indexes);
});
+
+ //Watching for functions
+ this.functions$ = this.store.select(s => s.functions.functions.results);
+ this.functions$.subscribe((data: any[]) => {
+ this.functions = data;
+ this.functionsFiltered = this.filter(this.functions);
+ });
+
+ // Watching for samples
+ this.sampleDataset$ = this.store.select(s => s.dataset.sample);
}
refreshMetadata() {
@@ -81,11 +144,24 @@
this.store.dispatch(new datasetActions.SelectDatasets('-'));
this.store.dispatch(new datatypesActions.SelectDatatypes('-'));
this.store.dispatch(new indexesActions.SelectIndexes('-'));
+ this.store.dispatch(new functionActions.SelectFunctions('-'));
}
dataverseFilter = {}
dataverseFilterMap = new Map();
+ createDatatypeDict(data) {
+ let newDict = new Object();
+
+ if (data) {
+ for (let i=0; i < data.length; i++) {
+ newDict[data[i].DataverseName + "." + data[i].DatatypeName] = i;
+ }
+ }
+
+ return newDict;
+ }
+
filter(data){
let results = [];
@@ -125,6 +201,7 @@
this.datasetsFiltered = this.filter(this.datasets);
this.datatypesFiltered = this.filter(this.datatypes);
this.indexesFiltered = this.filter(this.indexes);
+ this.functionsFiltered = this.filter(this.functions);
/* Open the dataset expansion panel if there is anything to show */
if (this.datasetsFiltered.length > 0) {
@@ -135,16 +212,213 @@
}
/*
+ * Traverse Metadata recursively, handles Metadata dataverse as well as regular dataverses
+ */
+ recursiveMetadataTraverse(data, toCreate): Object {
+ toCreate.DatatypeName = data.DatatypeName;
+ //primitive == no Derived field or Derived == undefined
+ if (data.Derived == undefined) {
+ //if primitive
+ toCreate.DatatypeName = data.DatatypeName;
+ toCreate.fields = [];
+ toCreate.primitive = true;
+ } else {
+ //if not primitive, need to check .Derived exists every time, or else handle primitive type
+ toCreate.DatatypeType = data.Derived.Tag;
+
+ //determine what type it is (Record, Ordered List or Unordered List). Ordered list and unordered list are handled the same
+ let list_type = "";
+
+ switch(toCreate.DatatypeType) {
+ case MetadataTypes.record:
+ toCreate.DatatypeType = MetadataTypesNames.record;
+ break;
+ case MetadataTypes.orderedList:
+ toCreate.DatatypeType = MetadataTypesNames.orderedList;
+ list_type = MetadataListTypes.orderedList;
+ break;
+ case MetadataTypes.unorderedList:
+ toCreate.DatatypeType = MetadataTypesNames.unorderedList;
+ list_type = MetadataListTypes.unorderedList;
+ break;
+ default:
+ break;
+ }
+
+ toCreate.fields = [];
+
+ if (data.Derived.Tag == "RECORD") {
+ // if it is a record, we must iterate over the fields and may have to recurse if there is a non primitive type
+ for (let field of data.Derived.Record.Fields) {
+ //if it is NOT a primitive type
+ if ((data.DataverseName + "." + field.FieldType) in this.datatypesDict &&
+ this.datatypes[this.datatypesDict[data.DataverseName + "." + field.FieldType]].Derived != undefined) {
+ field.Nested = true;
+
+ //get the nested object from datatypesDict
+ let nestedName = this.datatypesDict[data.DataverseName + "." + field.FieldType];
+ let nestedObject = this.datatypes[nestedName];
+
+ let nested_field = {
+ DatatypeName: field.FieldName,
+ DatatypeType: field.FieldType,
+ primitive: false,
+ fields: [],
+ isNullable: field.IsNullable,
+ isMissable: field.IsMissable,
+ OrderedList: false,
+ UnorderedList: false,
+ anonymous: nestedObject.Derived.IsAnonymous,
+ }
+
+ if (nestedObject.Derived.Tag == "RECORD") {
+ //object and should iterate over fields
+ field.NestedType = "Record";
+ field.NestedTypeType = nestedObject.DatatypeName;
+
+ let recurse_result = this.recursiveMetadataTraverse(nestedObject, {})
+
+ field.NestedRecord = recurse_result[0];
+
+ let toAdd = recurse_result[1];
+ toAdd.DatatypeType = "Record";
+ toAdd.primitive = false;
+ toAdd.anonymous = nestedObject.Derived.IsAnonymous;
+
+ nested_field.fields.push(toAdd);
+ }
+ else {
+ let listObject;
+ let nestedListType = "";
+ let nestedListTypeName = "";
+
+ //determine the type of list of the nested object
+ if (nestedObject.Derived.Tag == MetadataTypes.orderedList) {
+ nestedListType = MetadataListTypes.orderedList;
+ nestedListTypeName = MetadataTypesNames.orderedList;
+ } else {
+ nestedListType = MetadataListTypes.unorderedList;
+ nestedListTypeName = MetadataTypesNames.unorderedList;
+ }
+
+ nested_field[nestedListType] = true;
+ field.NestedType = nestedListTypeName;
+
+ if (data.DataverseName + "." + nestedObject.Derived[nestedListType] in this.datatypesDict) {
+ field.primitive = false;
+ listObject = this.datatypes[this.datatypesDict[data.DataverseName + "." + nestedObject.Derived[nestedListType]]];
+
+ let recurse_result = this.recursiveMetadataTraverse(listObject, {});
+
+ field.NestedRecord = recurse_result[0];
+
+ let toAdd = recurse_result[1];
+
+ if (toAdd.DatatypeType == nestedListTypeName) {
+ toAdd[nestedListType] = true;
+ } else {
+ toAdd[nestedListType] = false;
+ }
+
+ toAdd.primitive = false;
+ if (listObject.Derived != undefined)
+ toAdd.anonymous = listObject.Derived.IsAnonymous;
+
+ nested_field.fields.push(toAdd);
+ } else {
+ field.primitive = true;
+ nested_field.fields.push({
+ DatatypeName: nestedObject.Derived[nestedListType],
+ [nestedListType]: true,
+ primitive: true,
+ });
+ }
+ }
+
+ toCreate.fields.push(nested_field);
+ }
+ else {
+ field.Nested = false;
+ toCreate.fields.push({
+ DatatypeName: field.FieldName,
+ DatatypeType: field.FieldType,
+ primitive: true,
+ isMissable: field.IsMissable,
+ isNullable: field.IsNullable,
+ anonymous: false,
+ });
+ }
+ }
+ } else {
+ let listItem = this.datatypes[this.datatypesDict[data.DataverseName + "." + data.Derived[list_type]]];
+
+ toCreate[list_type] = true;
+
+ if (listItem == undefined) {
+ toCreate.fields.push({
+ DatatypeName: data.Derived[list_type],
+ [list_type]: true,
+ primitive: true,
+ anonymous: data.Derived.IsAnonymous,
+ })
+ } else {
+ let recurse_result = this.recursiveMetadataTraverse(listItem, {});
+
+ let toAdd = recurse_result[1];
+ toAdd.primitive = false;
+ if (listItem.Derived != undefined)
+ toAdd.anonymous = listItem.Derived.IsAnonymous;
+ toCreate.fields.push(toAdd);
+ }
+ }
+ }
+
+ return [data, toCreate];
+ }
+
+ /*
* opens the metadata inspector
*/
- openMetadataInspectorDialog(data): void {
+ openMetadataInspectorDialog(data, metadata_type): void {
let metadata = JSON.stringify(data, null, 8);
metadata = metadata.replace(/^{/, '');
metadata = metadata.replace(/^\n/, '');
metadata = metadata.replace(/}$/, '');
- let dialogRef = this.dialog.open(DialogMetadataInspector, {
- width: '500px',
- data: metadata,
+
+ //if metadata_type is dataset, sample said dataset and add to data to be displayed.
+
+ data.guts = metadata;
+
+ data.MetadataType = metadata_type
+ if (metadata_type == 'dataset') {
+ let dataset = "`" + data.DataverseName + "`" + "." + "`" + data.DatasetName + "`";
+ this.store.dispatch(new datasetActions.SampleDataset({dataset: dataset}));
+
+ this.sampleDataset$.subscribe((resp: any[]) => {
+ if (resp) {
+ this.dialogSamples[dataset] = JSON.stringify(resp[dataset], null, 2);
+ data.sample = this.dialogSamples[dataset];
+ } else {
+ data.sample = undefined;
+ }
+ });
+ }
+
+ //flatten data if datatype
+ if (metadata_type == 'datatype') {
+ let new_datatype = new Object();
+
+ let recurseResults = this.recursiveMetadataTraverse(data, new_datatype);
+ let converted = recurseResults[1];
+
+ data = recurseResults[0];
+
+ data.DataTypeTree = converted;
+ }
+
+ this.curr_dialogRef = this.dialog.open(DialogMetadataInspector, {
+ minWidth: '500px',
+ data: data,
hasBackdrop: false
});
}
@@ -158,9 +432,30 @@
export class DialogMetadataInspector {
constructor( public dialogCreateDsRef: MatDialogRef<DialogMetadataInspector>,
- @Inject(MAT_DIALOG_DATA) public data: any) { }
+ @Inject(MAT_DIALOG_DATA) public data: any) {
+ if (data.MetadataType == "datatype") {
+ this.dataSource.data = data.DataTypeTree.fields;
+ }
+ }
onClickClose() {
- this.dialogCreateDsRef.close();
+ this.dialogCreateDsRef.close(this.data['dialogID']);
}
-}
\ No newline at end of file
+
+ onClickJSON() {
+ this.showGuts = true;
+ this.hideJSONButton = true;
+ }
+
+ onClickParsed() {
+ this.showGuts = false;
+ this.hideJSONButton = false;
+ }
+
+ showGuts = false;
+ hideJSONButton = false
+
+ treeControl = new NestedTreeControl<DataTypeNode>(node => node.fields);
+ dataSource = new MatTreeNestedDataSource<DataTypeNode>();
+ hasChild = (_: number, node: DataTypeNode) => !!node.fields && node.fields.length > 0;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.html
index e95752e..89ed701 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.html
@@ -14,15 +14,16 @@
<mat-card class="sql-results-card">
<mat-card-content class="content-area">
<div class="divider">
- <div *ngIf='data.length != 0'>
- <tree-view [data]="data" [queryId]="queryId"></tree-view>
+ <div *ngIf="isError == false && hideOutput == false">
+ <tree-view [data]="data" [queryId]="queryId" [planFormat]="planFormat" [jsonPlan]="queryOptimizedLogicalPlan" [plan]="optimalLogicalPlan" [planName]="'OPTIMIZED PLAN'" [inputToOutput]="inputToOutput"></tree-view>
</div>
- <div *ngIf='queryOptimizedLogicalPlan != ""'>
- <plan-view [planFormat]="planFormat" [jsonPlan]="queryOptimizedLogicalPlan" [plan]="optimalLogicalPlan" [planName]="'OPTIMIZED PLAN'"></plan-view>
- </div>
+ <!--
+ Ignore Logical Plan for now...if users want back can add back in. However, current JSON->graph code does not support sub plans
+
<div *ngIf='queryLogicalPlan != ""'>
<plan-view [planFormat]="planFormat" [jsonPlan]="queryLogicalPlan" [plan]="logicalPlan" [planName]="'LOGICAL PLAN'"></plan-view>
</div>
+ -->
</div>
</mat-card-content>
-</mat-card>
\ No newline at end of file
+</mat-card>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.scss
index 67431c8..4949ded 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.scss
@@ -13,13 +13,14 @@
*/
$results-spacing-unit: 5px;
+
.sql-results-card {
+ display: block;
+ margin: 0 0px 0 0px;
+ padding: 0;
margin: ($results-spacing-unit);
- display: flex;
- flex-flow: column;
+ border-radius: 4px;
width: 100%;
- padding: 0px;
- border: none !important;
}
.content-area {
@@ -31,4 +32,4 @@
.divider {
width: 100%;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.ts
index 592e9b5..cd0abef 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/output.component.ts
@@ -11,7 +11,7 @@
See the License for the specific language governing permissions and
limitations under the License.
*/
-import { Component } from '@angular/core';
+import {Component, Input, OnChanges, SimpleChange} from '@angular/core';
import { Observable } from 'rxjs';
import { Store } from '@ngrx/store';
@@ -24,6 +24,10 @@
export class QueryOutputComponent {
+ @Input('inputToOutput') inputToOutput: Object;
+ @Input('isError') isError: boolean;
+ @Input('hideOutput') hideOutput: boolean;
+
data: any[];
currentQueryActive$: Observable < any > ;
currentQueryActive: string;
@@ -37,7 +41,14 @@
SQLresults: any;
queryId: any = "";
observedPlanFormat = "";
+ queryError: boolean;
+ currentQuery: any = 0;
+ ngOnChanges(changes: SimpleChange) {
+ this.inputToOutput = this.inputToOutput;
+ this.isError = this.isError;
+ this.hideOutput = this.hideOutput;
+ }
constructor(private store: Store <any>) {
let key = '1';
@@ -54,7 +65,7 @@
this.currentQueryActive = "0";
}
})
- /* this is the output when the quey runs for the first time */
+ /* this is the output when the query runs for the first time */
this.results$ = this.store.select(s => s.sqlQuery.sqlQueryResultHash);
this.results$.subscribe((data: any) => {
if (Object.keys(data).length !== 0 && data[this.currentQueryActive]) {
@@ -104,4 +115,4 @@
this.data = [];
}
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.html
deleted file mode 100644
index 5aa58d6..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.html
+++ /dev/null
@@ -1,79 +0,0 @@
-<!--/*
-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.
-*/-->
-<svg id='node{{level}}{{item}}{{subplan}}{{planName}}' xmlns="http://www.w3.org/2000/svg" xml:lang="en" xmlns:xlink="http://www.w3.org/1999/xlink"
- width="200px" height="160px" class="plan-node" (click)="seeDetails(viewParams_)">
- <title>{{details}}</title>
- <style>
- @keyframes cycle {
- 33.3% {
- visibility: visible;
- }
- 100% {
- visibility: hidden
- }
- }
-
- .lit {
- animation: cycle 9s step-start infinite;
- }
-
- .red .lit {
- animation-delay: -3s;
- }
-
- .yellow .lit {
- animation-delay: -6s;
- }
-
- .green .lit {
- animation-delay: 0;
- }
-
- .operation-text {
- font-size: 12px;
- fill: black;
- }
-
- .operation-see-more {
- font-size: 12px;
- fill: black;
- cursor: pointer;
- }
-
- .card {
- cursor: pointer;
- }
-
- .card:hover {
- stroke: blue;
- }
-
- .operation-details {
- visibility: none;
- transition: opacity 1s ease-in-out;
- opacity: 0;
- }
-
- </style>
- <text class="operation-text" x="50%" y="50%" text-anchor="middle">{{getNodeOperatorId()}} : {{getNodeName()}}</text>
-</svg>
-
-<div class="branch" *ngIf="node.inputs">
- <li *ngIf="checkSubPlan()" class="li sub">
- <plan-node-svg class="sub" [planName]="planName" [node]="node.inputs[item].subplan[0]" [level]="0" [item]="0" [subplan]="level+item+subplan+1"></plan-node-svg>
- </li>
- <li class="li" *ngFor="let subNode of node.inputs; let i = index">
- <plan-node-svg class="" [planName]="planName" [node]="subNode" [level]="level+1" [item]="i" [subplan]="subplan" [viewParams]="viewParams"></plan-node-svg>
- </li>
-</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.scss
deleted file mode 100644
index f1af051..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.scss
+++ /dev/null
@@ -1,230 +0,0 @@
-/*
-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.
-*/
-
-//vars
-$page-width: 1000px;
-$padding-base: 12px;
-$padding-sm: 5px;
-$padding-lg: 15px;
-$padding-xl: 25px;
-$font-size-base: 12px;
-$font-size-xs: round($font-size-base * 0.7);
-$font-size-sm: round($font-size-base * 0.9);
-$font-size-lg: round($font-size-base * 1.3);
-$font-size-xl: round($font-size-base * 1.7);
-$font-family-sans-serif: 'noto';
-$font-family-mono: 'source code';
-$line-height-base: 1.3;
-$gray-lightest: #f7f7f7;
-$gray-light: darken($gray-lightest, 10%);
-$gray: darken(#f7f7f7, 30%);
-$gray-dark: darken(#f7f7f7, 50%);
-$gray-darkest: darken($gray-lightest, 70%);
-$blue: #00B5E2;
-$dark-blue: #008CAF;
-$light-blue: #65DDFB;
-$red: #AF2F11;
-$dark-red: #7C210C;
-$light-red: #FB8165;
-$green: #279404;
-$yellow: #F8E400;
-$bg-color: $gray-lightest;
-$text-color: #4d525a;
-$text-color-light: lighten($text-color, 30%);
-$line-color: $gray-light;
-$line-color-light: lighten($gray-light, 10%);
-$link-color: $blue;
-$border-radius-base: 3px;
-$border-radius-lg: 6px;
-$main-color: $blue;
-$main-color-dark: $blue;
-$highlight-color: $blue;
-$highlight-color-dark: $dark-blue;
-$alert-color: #FB4418;
-$connector-height: 20px;
-$connector-line-small: 1px solid darken($line-color, 10%);
-$connector-line-big: 2px solid darken($line-color, 10%);
-.plan-nodea {
- display: table;
- position: relative;
- float: left;
-}
-
-.view-icon {
- font-size: 14px;
-}
-
-.flex-spacer {
- flex: 1 1 10%;
-}
-
-.dot {
- display: flex;
- flex-flow: column;
- margin-left: auto;
- margin-right: auto;
- padding: 0;
-}
-
-.plan-node {
- display: flex;
- flex-flow: column;
- justify-content: flex-start;
- color: $text-color;
- transition: hidden 0.8s;
- padding: 0px;
- font-size: 10px;
- border: 1px solid $line-color;
- margin: 0px;
- border-radius: $border-radius-base;
- width: 200px;
- height: 60px;
- box-shadow: 1px 1px 3px 0px rgba(0, 0, 0, 0.1);
- margin-left: auto;
- margin-right: auto;
- padding-left: auto;
- padding-right: auto;
- transition: height 0.3s ease-in-out;
- &:hover {
- border-color: $highlight-color;
- }
-}
-
-.plan {
- list-style: none !important;
- padding-bottom: $padding-lg * 3;
- margin-top: 0;
- padding-top: 0;
- padding-left: 0;
-
- .merge {
- display: flex;
- flex-flow: column;
- margin-top: 0 !important;
- list-style: none;
- padding-left: 0;
- color: #00B5E2 !important;
- border: 1px dashed #00B5E2;
- justify-content: center;
- align-items: center;
- }
-
- .branch {
- display: flex;
- flex-flow: row;
- margin-top: 0;
- list-style: none;
- padding-top: $connector-height;
- position: relative;
- transition: all 1s;
- padding-left: 0;
- color: black;
- margin-left: auto;
- margin-right: auto; // vertical
- &:before {
- content: '';
- position: absolute;
- top: 0;
- left: 50%;
- border-left: $connector-line-small;
- height: $connector-height;
- width: 0;
- color: black;
- margin-top: 0;
- }
- &:first-child {
- margin-top: 0;
- &:before {
- border: none;
- }
- }
- .branch {
- display: flex;
- flex-flow: row;
- margin-top: 0 !important;
- list-style: none;
- padding-left: 0;
- }
-
- li {
- display: inline;
- list-style-type: none;
- position: relative;
- padding: $connector-height $padding-sm 0 $padding-sm;
- transition: all 1s;
- margin-left: auto;
- margin-right: auto; // connectors
- &:before,
- &:after {
- content: '';
- position: absolute;
- top: 0;
- right: 50%;
- border-top: $connector-line-small;
- width: 50%;
- height: $connector-height;
- }
- &:after {
- right: auto;
- left: 50%;
- border-left: $connector-line-small;
- }
- &:only-child {
- padding-top: 0;
- &:after,
- &:before {
- display: none;
- }
- }
- &:first-child::before,
- &:last-child::after {
- border: 0 none;
- }
- &:last-child::before {
- border-right: $connector-line-small;
- border-radius: 0 $border-radius-lg 0 0;
- }
- &:first-child::after {
- border-radius: $border-radius-lg 0 0 0;
- }
- }
- }
-}
-
-.sub {
- .plan-node {
- background-color: rgb(230, 230, 230);
- }
-}
-
-.viewMe {
- display: inline-block;
- position: relative;
- cursor: pointer;
-}
-
-.node-summary {
- display: block;
- margin: 10px 0 10px 0;
- text-align: left;
-}
-
-.node-details {
- display: block;
- text-align: left;
-}
-
-.node-subplan {
- float: right;
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.ts
deleted file mode 100644
index a8d1987..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-node-svg.component.ts
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
-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.
-*/
-/*
-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.
-*/
-import { Renderer2, ViewEncapsulation, Component, Input } from '@angular/core';
-
-export interface ViewParams {
- viewMode: string,
- width: string,
- height: string,
- visible: string,
- display: string,
- opacity: number,
- border: string
-}
-
-export const FULL:ViewParams = {
- viewMode: 'FULL',
- width: '350px',
- height: '180px',
- visible: 'visible',
- display: 'block',
- opacity: 1,
- border: "2px solid #0000FF"
-};
-
-export const NORMAL:ViewParams = {
- viewMode: 'NORMAL',
- width: '200px',
- height: '60px',
- visible: 'hidden',
- display: 'none',
- opacity: 0,
- border: "none"
-};
-
-@Component({
- moduleId: module.id,
- selector: 'plan-node-svg',
- templateUrl: 'plan-node-svg.component.html',
- styleUrls: ['plan-node-svg.component.scss'],
- encapsulation: ViewEncapsulation.None,
-})
-
-export class PlanNodeSVGComponent {
- @Input() node: any;
- @Input() level;
- @Input() item = 0;
- @Input() subplan = 0;
- @Input() planName = "";
- @Input() viewParams;
-
- details: any;
- viewParams_: any;
-
- constructor(private renderer: Renderer2) {}
-
- numberOfInputs: 0;
- selected = false;
-
- ngOnInit() {
-
- this.viewParams_ = NORMAL;
-
- /* Some preprocessing to show explanation details */
- if (this.node.inputs){
- this.numberOfInputs = this.node.inputs.length;
- } else {
- this.numberOfInputs = 0;
- }
-
- if (this.node) {
- let node_= JSON.parse(JSON.stringify(this.node));
-
- if (node_.inputs) {
- delete node_['inputs'];
- }
-
- if (node_.subplan) {
- delete node_['subplan'];
- }
-
- if (node_.visible != undefined ) {
- delete node_['visible'];
- }
-
- if (node_.viewDetails != undefined) {
- delete node_['viewDetails'];
- }
-
- if (node_.operator) {
- delete node_['operator'];
- }
-
- if (node_.operatorId) {
- delete node_['operatorId'];
- }
-
- this.details = JSON.stringify(node_, null, 8);
-
- this.details = this.details.replace(/^{/, '');
- this.details = this.details.replace(/^\n/, '');
- this.details = this.details.replace(/}$/, '');
- }
- }
-
- getNodeName() {
- if(this.node) {
- if (this.node.operator) {
- return (this.node.operator).toUpperCase();
- } else {
- return "NA";
- }
- }
- }
-
- getNodeOperatorId() {
- if(this.node) {
- if (this.node.operatorId) {
- return (this.node.operatorId).toUpperCase();
- } else {
- return "NA";
- }
- }
- }
-
- getNodeSubPlan() {
- if(this.node) {
- if (this.node['inputs']) {
- if (this.node['inputs'][this.item]) {
- if (this.node['inputs'][this.item]['subplan']) {
- return "Subplan";
- } else {
- return "";
- }
- } else {
- return "";
- }
- }
- }
- }
-
- seeDetails(me) {
- // Future Implementation
- }
-
- checkSubPlan() {
- if(this.node) {
- if (this.node['inputs']) {
- if (this.node['inputs'][this.item]) {
- if (this.node['inputs'][this.item]['subplan']) {
- return true;
- } else {
- return false;
- }
- } else {
- return false;
- }
- } else {
- return false;
- }
- }
- }
-
- checkMerge() {
- if(this.node) {
- if (this.node['mergeWith']) {
- return true;
- } else {
- return false;
- }
- }
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.html
deleted file mode 100644
index 6eaee72..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.html
+++ /dev/null
@@ -1,37 +0,0 @@
-<!--/*
-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.
-*/ -->
-<div *ngIf="plan_" class="plan-graph" id={{planName}}>
- <mat-expansion-panel hideToggle [expanded]="true">
- <mat-expansion-panel-header class="plan-header header-centered-v">
- <mat-icon>assessment</mat-icon>
- <mat-panel-title>{{planName}}</mat-panel-title>
- <mat-panel-description></mat-panel-description>
- </mat-expansion-panel-header>
- <mat-panel-description class='content'>
- <div *ngIf="!jsonButtonDisabled" class='panel'>
- <button id='jsonButton' mat-button class='button' (click)="showJSON()" matTooltip="Toggle JSON or Graphic View">JSON</button>
- </div>
- <div class="divider">
- <div *ngIf="!jsonVisible" class="plan">
- <plan-node-svg [planName]="planName" [node]="plan_" [level]="0" [item]="0" [subplan]="0"></plan-node-svg>
- </div>
- <div *ngIf="jsonVisible" class="json">
- <div class='center'>
- <pre class="json-content">{{jsonPlan}}</pre>
- </div>
- </div>
- </div>
- </mat-panel-description>
- </mat-expansion-panel>
-</div>
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.scss
deleted file mode 100644
index 28dd380..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.scss
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
-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.
-*/
-
-.plan-graph {
- display: block;
- margin: 0 0px 0 0px;
- padding: 0;
- mat-expansion-panel {
- border: none !important;
- }
-}
-
-.panel {
- display: flex;
- flex-flow: row;
- justify-content: flex-start;
- border-bottom: 1px dashed gray;
- margin-bottom: 15px;
-}
-
-.plan-header {
- max-height: 42px;
- min-height: 42px;
- font-size: 0.80rem;
- font-weight: 500;
- border-bottom: 1px solid gray;
-}
-
-.divider {
- display: flex;
- flex-flow: row;
- align-items: flex-start;
-}
-
-.plan {
- margin: 0px;
- padding: 20px;
- padding-left: 50px;
- padding-right: 50px;
- margin-right: auto;
- margin-left: auto;
- padding-left: 50px;
- overflow: visible;
-}
-
-.plan1 {
- display: flow;
- flex-flow: row;
-}
-
-.content {
- margin-top: 20px;
- display: block;
- font-size: 0.80rem;
- font-weight: 500;
-}
-
-.json {
- //padding: 20px;
- //padding-right: 50px;
- //margin-right: 25px;
- min-width: 100%;
- max-width: 100%;
-}
-
-.json-content {
- //border-left: 1px solid gray;
- padding-left: 25px; //display: inline-block;
- //margin: auto;
-}
-
-.button {
- font-size: 12px !important;
- float: right;
- color: blue !important;
- margin-bottom: 15px;
-}
-
-#wrapper {
- position: relative;
-}
-
-.center {
- margin-left: auto;
- margin-right: auto;
- width: 50%;
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.ts
deleted file mode 100644
index dd0ee6b..0000000
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-view.component.ts
+++ /dev/null
@@ -1,115 +0,0 @@
-import { Component, Input, SimpleChange } from '@angular/core';
-
-export interface planCount {
- nodesCnt: number,
- levelsCnt: number
-}
-
-@Component({
- selector: 'plan-view',
- templateUrl: 'plan-view.component.html',
- styleUrls: ['plan-view.component.scss'],
-})
-
-export class PlanViewComponent {
-
- @Input() planFormat: any;
- @Input() plan: any;
- @Input() planName: any;
- @Input() jsonPlan: any;
-
- plan_: any;
- numberOfLevels: number = 0;
- numberOfNodes: number = 0;
- jsonVisible = false;
- jsonButtonDisabled = false;
-
- constructor() {}
-
- ngOnInit() {}
-
- ngOnChanges() {
- this.plan_ = this.plan;
- /* If plan format is JSON analyze and augment for visualization */
- if (this.planFormat === 'JSON') {
- let summary : planCount = {nodesCnt:0, levelsCnt:0}
- summary = this.analyzePlan(this.plan_, summary);
- this.numberOfLevels = summary.levelsCnt;
- this.numberOfNodes = summary.nodesCnt;
- this.jsonVisible = false;
- this.jsonButtonDisabled = false;
- } else {
- this.jsonVisible = true;
- this.jsonButtonDisabled = true;
- }
- }
-
- /*
- * See the JSON contents inside of each node
- */
- showJSON() {
- this.jsonVisible = !this.jsonVisible;
- }
-
- /*
- * Check the merge paths, from operation ID
- */
- operation = [];
- checkOperationId(operationId, levelsCnt){
- console.log('LEVEL:' + levelsCnt + 'OP' + operationId)
- // console.log(this.operation)
- if (this.operation.length > 0) {
- for (let i = 0; i < this.operation.length; i++) {
- if (this.operation[i] === operationId) {
- console.log('found')
- console.log('BREAK')
- this.operation = [];
- return true;
- }
- }
- }
- this.operation.push(operationId);
- console.log('not found')
- return false;
- }
-
- /*
- * Counts the number of nodes/operations in the tree
- */
- analyzePlan(plan, planCounter) {
- planCounter.nodesCnt += 1;
- planCounter.levelsCnt += 1;
- let nodes = {}
- nodes = plan;
- // augment
- if (nodes) {
- nodes['visible'] = true;
- nodes['viewDetails'] = false;
- if (nodes['inputs']) {
- for (let i = 0; i< nodes['inputs'].length; i++)
- {
- planCounter = this.analyzePlan(nodes['inputs'][i], planCounter);
- }
- }
- }
- return planCounter;
- }
-
- /*
- * See the JSON contents inside of each node, with pre-format
- * Not used in this version
- */
- toggleViewDetails(plan) {
- let nodes = {}
- nodes = plan;
- // augment
- nodes['visible'] = true;
- nodes['viewDetails'] = !nodes['viewDetails'];
- if (nodes['inputs']) {
- for (let i = 0; i< nodes['inputs'].length; i++)
- {
- this.toggleViewDetails(nodes['inputs'][i]);
- }
- }
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
new file mode 100644
index 0000000..a131353
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.html
@@ -0,0 +1,184 @@
+<!--/*
+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.
+*/ -->
+<div *ngIf="plan_" class="plan-graph" id={{planName}}>
+ <div *ngIf="planFormat=='JSON'" class='panel' #navBar [class.sticky]="sticky" [class.collapsed]="collapsed">
+ <div class="plan-explorer-title">
+ <h2 class="heading-title">Plan Explorer</h2>
+ </div>
+ <div class="orientation">
+ <mat-form-field>
+ <h3 class="heading-title">View:</h3>
+ <mat-select [(ngModel)]="planOrientation" (ngModelChange)="setOrientation($event)">
+ <mat-option *ngFor="let orientation of orientations" [value]="orientation.value">
+ {{orientation.label}}
+ </mat-option>
+ </mat-select>
+ </mat-form-field>
+ </div>
+ <button mat-raised-button class="zoomFit-button other-buttons" (click)="fitGraph()" matTooltip="Zoom out to Fit Whole Plan">Zoom To Fit</button>
+ <h3 class="heading-title">Navigation:</h3>
+ <mat-checkbox class="detailed-checkbox" [checked]="detailed" (change)="setDetail($event.checked)">Detailed</mat-checkbox>
+ <h4 class="heading-title">Node:</h4>
+ <div>
+ <div class="node-nav">
+ <mat-form-field>
+ <mat-select [(ngModel)]="selectedNode" (ngModelChange)="panToNode($event)">
+ <mat-option *ngFor="let node of nodeIdsArr" [value]="node.value">
+ {{node.label}}
+ </mat-option>
+ </mat-select>
+ </mat-form-field>
+ <div class="node-mover">
+ <button mat-icon-button class="inc-dec" (click)="decrementNode()" matTooltip="Move Up a Node" [disabled]="nodeIdx == 0"><mat-icon>keyboard_arrow_up</mat-icon></button>
+ <button mat-icon-button class="inc-dec" (click)="incrementNode()" matTooltip="Move Down a Node" [disabled]="nodeIdx == nodesArr.length - 1"><mat-icon>keyboard_arrow_down</mat-icon></button>
+ </div>
+ </div>
+ </div>
+ <h4 class="heading-title" *ngIf="detailed">Variable:</h4>
+ <div class="variable-nav" *ngIf="detailed">
+ <div class="variable-select-move">
+ <mat-form-field class="variable-select">
+ <mat-label *ngIf="selectedVariableOccurrences == undefined">See Variable Occurrences</mat-label>
+ <mat-label *ngIf="selectedVariableOccurrences">Variable Occurrences ({{this.occurrenceArrayIdx+1}}/{{this.selectedVariableOccurrencesArray.length}})</mat-label>
+ <mat-select [(ngModel)]="selectedVariableOccurrences" (ngModelChange)="setSelectedVariableOccurrences($event)">
+ <mat-option *ngFor="let variable of variables" [value]="variable">
+ {{variable}}
+ </mat-option>
+ </mat-select>
+ </mat-form-field>
+ <div class="node-mover">
+ <button mat-icon-button class="inc-dec" [disabled]="!selectedVariableOccurrences" (click)="decrementOccurrence()" matTooltip="Move Up an Occurrence"><mat-icon>keyboard_arrow_up</mat-icon></button>
+ <button mat-icon-button class="inc-dec" [disabled]="!selectedVariableOccurrences" (click)="incrementOccurrence()" matTooltip="Move Down an Occurrence"><mat-icon>keyboard_arrow_down</mat-icon></button>
+ </div>
+ </div>
+ <div class="declaration-back-btns">
+ <button mat-raised-button class="other-buttons" [disabled]="!selectedVariableOccurrences"(click)="jumpToDeclaration()" matTooltip="Skip to Declaration">DECLARATION</button>
+ <button mat-raised-button class="undo-buttons" matTooltip="Jump back to Previous Node" [disabled]="previousNodeId == undefined" (click)="jumpBack()">BACK</button>
+ </div>
+ </div>
+ <h4 class="heading-title" *ngIf="detailed">Search:</h4>
+ <div class="search" *ngIf="detailed">
+ <div class="search-nav">
+ <mat-form-field>
+ <mat-label *ngIf="!matchesFound">Search</mat-label>
+ <mat-label *ngIf="matchesFound">Matches ({{this.matchIdx+1}}/{{this.searchMatches.length}})</mat-label>
+ <input matInput [(ngModel)]="searchRegex">
+ </mat-form-field>
+ <div class="node-mover">
+ <button mat-icon-button class="inc-dec" [disabled]="!matchesFound" (click)="decrementMatch()" matTooltip="Move up a Match"><mat-icon>keyboard_arrow_up</mat-icon></button>
+ <button mat-icon-button class="inc-dec" [disabled]="!matchesFound" (click)="incrementMatch()" matTooltip="Move down a Match"><mat-icon>keyboard_arrow_down</mat-icon></button>
+ </div>
+ </div>
+ <button class="search-btn" mat-raised-button (click)="onClickSearch()" [disabled]="searchRegex == ''">SEARCH</button>
+ </div>
+ <button class="clear-btn" mat-raised-button (click)="clearSelections()">CLEAR</button>
+ </div>
+ <div class="divider">
+ <div *ngIf="!jsonVisible" class="plan">
+ <ngx-graph
+ layout="dagre"
+ [view]="[850,800]"
+ [showMiniMap]="true"
+ [zoomToFit$]="zoomToFit$"
+ [center$]="center$"
+ [panToNode$]="panToNode$"
+ [enableZoom]="true"
+ [links]="edgesArr"
+ [nodes]="nodesArr"
+ [draggingEnabled]="false"
+ [zoomSpeed]="0.025"
+ [update$]="update$"
+ [layoutSettings]="{
+ orientation: planOrientation,
+ alignment: 'C',
+ nodePadding: 250,
+ rankPadding: 50
+ }">
+ <ng-template #defsTemplate>
+ <svg:marker id="arrow" viewBox="0 -5 10 10" refX="8" refY="0" markerWidth="5" markerHeight="5" orient="auto">
+ <svg:path d="M0,-5L10,0L0,5" class="arrow-head" />
+ </svg:marker>
+ </ng-template>
+
+ <ng-template #nodeTemplate let-node>
+ <svg:g class="node">
+ <svg:rect *ngIf="!node.selected"
+ [attr.width]="node.dimension.width"
+ [attr.height]="node.dimension.height"
+ [attr.rx] = "15"
+ [attr.fill]="node.color"
+ fill-opacity="0.2"
+ stroke="black"
+ stroke-width="1.5"
+ />
+ <svg:rect *ngIf="node.selected"
+ [attr.width]="node.dimension.width"
+ [attr.height]="node.dimension.height"
+ [attr.rx] = "15"
+ [attr.fill] = "node.color"
+ fill-opacity="0.0"
+ stroke="black"
+ stroke-width="4.5"
+ />
+ <svg:rect *ngIf="node.selected"
+ [attr.width]="node.dimension.width"
+ [attr.height]="node.dimension.height"
+ [attr.rx] = "15"
+ [attr.fill]="node.color"
+ fill-opacity="0.2"
+ stroke="red"
+ stroke-width="3"
+ />
+ <svg:text *ngIf="!detailed" dominant-baseline="middle" text-anchor="middle" [attr.y]="node.dimension.height / 2" [attr.x]="node.dimension.width / 2" >
+ {{node.label}}
+ </svg:text>
+
+ <svg:text *ngIf="detailed" alignment-baseline="top" text-anchor="middle">
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+
+ </svg:tspan>
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+ {{node.detailed_label}}
+ </svg:tspan>
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+ {{node.physical_operator}}
+ </svg:tspan>
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+ {{node.execution_mode}}
+ </svg:tspan>
+ <ng-container *ngFor="let details of node.details | keyvalue">
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+ {{details.value}}
+ </svg:tspan>
+ </ng-container>
+ <svg:tspan [attr.x]="node.dimension.width / 2" dy="1.5em">
+ _
+ </svg:tspan>
+ </svg:text>
+ </svg:g>
+ </ng-template>
+ <ng-template #linkTemplate let-link>
+ <svg:g class="edge">
+ <svg:path class="line" stroke-width="2" marker-end="url(#arrow)"></svg:path>
+ </svg:g>
+ </ng-template>
+ </ngx-graph>
+ </div>
+ <div *ngIf="jsonVisible" class="json">
+ <div class='center'>
+ <pre class="json-content">{{jsonPlan}}</pre>
+ </div>
+ </div>
+ </div>
+</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
new file mode 100644
index 0000000..6c40a68
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.scss
@@ -0,0 +1,231 @@
+/*
+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.
+*/
+
+/*
+.plan-graph {
+ display: block;
+ margin: 0 0px 0 0px;
+ padding: 0;
+ width: 100%;
+ mat-expansion-panel {
+ border: none !important;
+ }
+}
+ */
+
+.plan-graph {
+ display: flex;
+ flex-flow: row;
+ margin: 0 0px 0 0px;
+ padding: 0;
+ width: 100%;
+ mat-expansion-panel {
+ border: none !important;
+ }
+}
+
+.panel {
+ order: 2;
+ display: flex;
+ flex-flow: column;
+ justify-content: stretch;
+ align-items: flex-start;
+ border: 1.5px solid grey;
+ border-radius: 15px;
+ margin-left: auto;
+ height: 100%;
+}
+
+.panel > * {
+ margin-left: 10px;
+ margin-right: 10px;
+}
+
+.plan-explorer-title {
+ width: 100%;
+ text-align: center;
+ margin-left: 0px !important;
+ margin-right: 0px !important;
+}
+
+.heading-title {
+ color: blue;
+ margin-bottom: 5px;
+ margin-top: 5px;
+}
+
+.mat-icon-button.inc-dec {
+ width: 25px;
+ height: 25px;
+ line-height: 25px;
+
+ .mat-icon {
+ font-size: 25px;
+ width: 25px;
+ height: 25px;
+ line-height: 25px;
+ }
+}
+
+.plan-header {
+ max-height: 42px;
+ min-height: 42px;
+ font-size: 0.80rem;
+ font-weight: 500;
+ border-bottom: 1px solid gray;
+}
+
+.divider {
+ display: flex;
+ flex-flow: row;
+ align-items: flex-start;
+}
+
+.plan {
+ margin: 0px;
+ padding: 20px;
+ padding-left: 50px;
+ padding-right: 50px;
+ margin-right: auto;
+ margin-left: auto;
+ padding-left: 50px;
+ min-height: 800px;
+ max-height: 1500px;
+ border: 1.5px solid grey;
+ border-radius: 15px;
+}
+
+.variable-nav {
+ display: flex;
+ flex-flow: column;
+}
+
+.declaration-back-btns {
+ display: flex;
+ flex-flow: row;
+ justify-content: space-between;
+}
+
+.variable-select-move {
+ display: flex;
+ flex-flow: row;
+ justify-content: flex-start;
+}
+
+.search-nav {
+ display: flex;
+ flex-flow: row;
+ justify-content: flex-start;
+}
+
+.search {
+ display: flex;
+ flex-flow: column;
+ align-items: flex-start;
+}
+
+.search-btn {
+ align-self: center;
+ width: 100%;
+}
+
+.node-nav {
+ display: flex;
+ flex-float: row;
+}
+
+.node-mover {
+ display: flex;
+ flex-flow: column;
+ overflow: auto;
+}
+
+
+.plan1 {
+ display: flow;
+ flex-flow: row;
+}
+
+.content {
+ margin-top: 20px;
+ display: block;
+ font-size: 0.80rem;
+ font-weight: 500;
+}
+
+.json {
+ //padding: 20px;
+ //padding-right: 50px;
+ //margin-right: 25px;
+ min-width: 100%;
+ max-width: 100%;
+}
+
+.json-content {
+ //border-left: 1px solid gray;
+ padding-left: 25px; //display: inline-block;
+ //margin: auto;
+}
+
+.button {
+ font-size: 12px !important;
+ float: right;
+ color: blue !important;
+ margin-bottom: 15px;
+}
+
+#wrapper {
+ position: relative;
+}
+
+.center {
+ margin-left: auto;
+ margin-right: auto;
+ width: 50%;
+}
+
+.sticky {
+ position: fixed;
+ top: 10px;
+ left: 50%;
+ background-color: white;
+ border: 1px solid grey;
+ border-radius: 5px;
+ width: auto;
+ transform: translateX(-50%);
+}
+
+
+.collapsed {
+ width: auto;
+}
+
+.minimize-btn {
+ align-self: flex-start;
+}
+
+.clear-btn {
+ margin-top: 10px;
+ margin-bottom: 10px;
+ align-self: center;
+}
+
+.orientation {
+ display: flex;
+ flex-flow: column;
+}
+
+.zoomFit-button {
+ align-self: center;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.ts
new file mode 100644
index 0000000..4dcee8a
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/plan-viewer.component.ts
@@ -0,0 +1,681 @@
+/*
+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.
+*/
+import {Component, Input, SimpleChange, HostListener, ViewChild, ElementRef} from '@angular/core';
+import { Subject } from "rxjs";
+
+export interface planCount {
+ nodesCnt: number,
+ levelsCnt: number
+}
+
+@Component({
+ selector: 'plan-viewer',
+ templateUrl: 'plan-viewer.component.html',
+ styleUrls: ['plan-viewer.component.scss'],
+})
+
+export class PlanViewerComponent {
+
+ @Input() planFormat: any;
+ @Input() plan: any;
+ @Input() planName: any;
+ @Input() jsonPlan: any;
+
+ plan_: any;
+ jsonVisible = false;
+ detailed: boolean = false;
+ nodesArr: any[];
+ nodeIdsArr: any[] = [];
+ ids: any[] = [];
+ edgesArr: any[];
+
+ //search variables
+ flatJSONGraph: any = {};
+ searchRegex: string = "";
+ searchMatches: any[] = [];
+ matchesFound: boolean = false;
+ matchIdx: number = 0;
+
+ //variables for ngx-graph
+ zoomToFit$: Subject<boolean> = new Subject();
+ center$: Subject<boolean> = new Subject();
+ update$: Subject<boolean> = new Subject();
+ panToNode$: Subject<any> = new Subject();
+
+ //drop down variables
+ planOrientation = "BT";
+ selectedNode = "n11";
+
+ previousNodeId: any;
+ previouseOccurrenceArrayIdx: number;
+
+ selectedVariableOccurrences: any;
+ selectedVariableOccurrencesArray: any[];
+ occurrenceArrayIdx: number = 0;
+
+ nodeIdx = 0;
+ orientations: any[] = [
+ {
+ label: "Bottom to Top",
+ value: "BT"
+ },
+ {
+ label: "Top to Bottom",
+ value: "TB"
+ }
+ /*
+ Left to Right or Right to Left do not look right yet
+
+ {
+ label: "Left to Right",
+ value: "LR"
+ },
+ {
+ label: "Right to Left",
+ value: "RL"
+ }
+ */
+ ];
+
+ colors = [
+ "#00ff00",
+ "#0000ff",
+ "#ffff00",
+ "#8b008b",
+ "#ffa500",
+ "#ee82ee",
+ "#ff0000",
+ "#9acd32",
+ "#20b2aa",
+ "#00fa9a",
+ "#db7093",
+ "#eee8aa",
+ "#6495ed",
+ "#ff1493",
+ "#ffa07a",
+ "#2f4f4f",
+ "#8b4513",
+ "#006400",
+ "#808000",
+ "#483d8b",
+ "#000080"
+ ]
+
+ coloredNodes: any = {};
+
+ variablesOccurrences: any = {};
+ variablesDeclarations: any = {};
+ variables: any[];
+
+ constructor() {}
+
+ ngOnInit() {}
+
+ ngAfterViewInit() {
+ }
+
+ ngOnChanges() {
+ this.plan_ = this.plan;
+
+ /* If plan format is JSON analyze and augment for visualization */
+ if (this.planFormat === 'JSON') {
+ //clear previous plans results
+ this.nodesArr = [];
+ this.edgesArr = [];
+ this.nodeIdsArr = [];
+ this.ids = [];
+ this.selectedNode = 'n11';
+
+ this.previousNodeId = undefined;
+ this.previouseOccurrenceArrayIdx = undefined;
+
+ this.selectedVariableOccurrences = undefined;
+ this.selectedVariableOccurrencesArray = undefined;
+ this.occurrenceArrayIdx = 0;
+
+ this.nodeIdx = 0;
+
+ this.coloredNodes = {};
+
+ this.variablesOccurrences = {};
+ this.variablesDeclarations = {};
+ this.variables = undefined;
+
+ this.searchRegex = "";
+ this.searchMatches = [];
+ this.matchesFound = false;
+ this.matchIdx = 0;
+
+ let nodesSet = new Set();
+ let edgesSet = new Set();
+
+ let recurseResults = this.createLinksEdgesArray(this.plan_, this.nodesArr, this.edgesArr, nodesSet, edgesSet);
+
+ this.nodesArr = recurseResults[0];
+ this.edgesArr = recurseResults[1];
+
+ this.variables = Object.keys(this.variablesOccurrences);
+
+ //get declarations from variableOccurrences
+ for (let variable of this.variables) {
+ //extract first occurrence of variable (last because we parse from bottom->down)
+ this.variablesDeclarations[variable] = this.variablesOccurrences[variable][this.variablesOccurrences[variable].length-1];
+ }
+
+ this.jsonVisible = false;
+ } else {
+ this.jsonVisible = true;
+ }
+ }
+
+ /*
+ Function that makes the entire graph to fit the view
+ */
+ fitGraph() {
+ this.zoomToFit$.next(true);
+ this.center$.next(true);
+ }
+
+ /*
+ * Create links array and edges array for NgxGraphModule
+ */
+ createLinksEdgesArray(plan, nodesArr, edgesArr, nodesSet, edgesSet) {
+ let nodes = {};
+ nodes = plan;
+
+ let nodeToAdd = {};
+
+ if (nodes) {
+ if (!nodesSet.has(nodes['operatorId'])) {
+ nodesSet.add(nodes['operatorId']);
+ nodeToAdd['id'] = "n" + nodes['operatorId'];
+ nodeToAdd['id'] = nodeToAdd['id'].replace(/\./g, '');
+
+ //read variables and expressions from node and add to this.nodeOccurrences
+ this.storeVariablesExpressions(nodes, nodeToAdd['id']);
+
+ this.flatJSONGraph[nodeToAdd['id']] = "";
+
+ //logic for label
+ nodeToAdd['label'] = nodes['operatorId'] + " : " + nodes['operator'];
+
+ nodeToAdd['detailed_label'] = nodes['operatorId'] + " : " + nodes['operator'];
+ nodeToAdd['physical_operator'] = nodes['physical-operator'];
+ nodeToAdd['execution_mode'] = "[" + nodes['execution-mode'] + "]"
+
+ nodeToAdd["details"] = {};
+
+ nodeToAdd['selected'] = false;
+ nodeToAdd['operator'] = nodes['operator'];
+
+ //case for having both expressions and variables
+ if (nodes['expressions'] && nodes['variables']) {
+ nodeToAdd['detailed_label'] += `${this.variableExpressionStringify(nodes['expressions'])} <- ${this.variableExpressionStringify(nodes['variables'])}`;
+ }
+ //case for having only expressions
+ if (nodes['expressions'] && nodes['variables'] == undefined) {
+ nodeToAdd['detailed_label'] += this.variableExpressionStringify(nodes['expressions']);
+ }
+
+ //case for having only variables
+ if (nodes['variables'] && nodes['expressions'] == undefined) {
+ //if data scan, different
+ if (nodes['data-source']) {
+ nodeToAdd['details']['data-scan'] = `[]<-${this.variableExpressionStringify(nodes['variables'])}<-${nodes['data-source']}`;
+ }
+ //else
+ else
+ nodeToAdd['detailed_label'] += `(${this.variableExpressionStringify(nodes['variables'])})`;
+ }
+
+ //limit value
+ if (nodes['value']) {
+ nodeToAdd['detailed_label'] += ` ${nodes['value']}`;
+ }
+
+ //group by operator group-by list
+ if (nodes['group-by-list'])
+ nodeToAdd['details']['group_by_list'] = `group by ([${this.groupByListStringify(nodes['group-by-list'])}])`;
+
+ //group by operator decor-list
+ if (nodes['decor-list'])
+ nodeToAdd['details']['decor_list'] = `decor ([${this.groupByListStringify(nodes['decor-list'])}])`;
+
+ //join operator condition
+ if (nodes['condition']) {
+ nodeToAdd['details']['condition'] = `join condition (${nodes['condition']})`;
+ }
+
+ let nodeDropDown = {};
+ nodeDropDown['label'] = nodeToAdd['label'];
+ nodeDropDown['value'] = nodeToAdd['id'];
+ this.nodeIdsArr.push(nodeDropDown);
+
+ for (let val of Object.values(nodeToAdd)) {
+ this.flatJSONGraph[nodeToAdd['id']] += String(val).toLowerCase() + " ";
+ }
+
+ //Dynamic node coloring
+ if (nodeToAdd['operator'] in this.coloredNodes) {
+ nodeToAdd['color'] = this.coloredNodes[nodeToAdd['operator']];
+ } else {
+ if (this.colors.length > 1) {
+ let nodeColor = this.colors[0];
+ this.colors.splice(0, 1);
+ nodeToAdd['color'] = nodeColor;
+
+ this.coloredNodes[nodeToAdd['operator']] = nodeColor;
+ } else {
+ let nodeColor = "#ffffff";
+ nodeToAdd['color'] = nodeColor;
+
+ this.coloredNodes[nodeToAdd['operator']] = nodeColor;
+ }
+ }
+
+ this.ids.push(nodeToAdd['id']);
+
+ nodesArr.push(nodeToAdd);
+ }
+
+ if (nodes['inputs']) {
+ for (let i = 0; i < nodes['inputs'].length; i++) {
+ let edge = nodes['operatorId'].slice() + "to" + nodes['inputs'][i]['operatorId'].slice();
+ edge = edge.replace(/\./g, '');
+
+ if (!edgesSet.has(edge)) {
+ edgesSet.add(edge);
+
+ //create the edge
+ let edgeToAdd = {};
+
+ edgeToAdd['id'] = "e" + edge;
+ edgeToAdd['source'] = "n" + nodes['inputs'][i]['operatorId'];
+ edgeToAdd['source'] = edgeToAdd['source'].replace(/\./g, '');
+ edgeToAdd['target'] = "n" + nodes['operatorId'];
+ edgeToAdd['target'] = edgeToAdd['target'].replace(/\./g, '');
+
+ edgesArr.push(Object.assign(edgeToAdd, {}));
+ }
+
+
+ let recurseResult = this.createLinksEdgesArray(nodes['inputs'][i], nodesArr, edgesArr, nodesSet, edgesSet);
+
+ nodesArr = recurseResult[0];
+ edgesArr = recurseResult[1];
+ nodesSet = recurseResult[2];
+ edgesSet = recurseResult[3];
+
+ }
+ }
+ }
+
+ return [nodesArr, edgesArr, nodesSet, edgesSet];
+ }
+
+ /*
+ * Extracts variables and expressions and stores occurences in this.variablesOccurrences
+ */
+ storeVariablesExpressions(node, id) {
+ if (node['expressions']) {
+ if (node['operator'] == 'assign') {
+ for (let expression of node['expressions']) {
+ let matches = expression.match(/\$\$.*?(?=,|\])/g)
+
+ if (matches) {
+ for (let match of matches) {
+ this.addVariableExpression(match, id);
+ }
+ }
+ }
+ } else {
+ for (let expression of node['expressions']) {
+ this.addVariableExpression(expression, id);
+ }
+ }
+ }
+ if (node['variables']) {
+ for (let variable of node['variables']) {
+ this.addVariableExpression(variable, id);
+ }
+ }
+ if (node['group-by-list']) {
+ for (let item of node['group-by-list']) {
+ this.addVariableExpression(item.variable, id);
+ this.addVariableExpression(item.expression, id);
+ }
+ }
+ if (node['decor-list']) {
+ for (let item of node['decor-list']) {
+ this.addVariableExpression(item.variable, id);
+ this.addVariableExpression(item.expression, id);
+ }
+ }
+ if (node['condition'] || node['operator'] == 'exchange') {
+ //does this for joins or exchanges (HASH_PARTITION_EXCHANGE contains variables/expressions)
+ //regex extracts variables / expressions ($$ match until a ',' ']', or '('
+ let matches = node['physical-operator'].match(/\$\$.*?(?=,|\]|\()/g)
+
+ if (matches) {
+ for (let match of matches) {
+ this.addVariableExpression(match, id);
+ }
+ }
+ }
+ }
+
+ /*
+ * Helper function that creates a set if var/exp not in this.variableOccurrences, then stores the id of the node
+ */
+ addVariableExpression(varExp, id) {
+ if (!(varExp in this.variablesOccurrences)) {
+ this.variablesOccurrences[varExp] = [];
+ }
+ if (!(this.variablesOccurrences[varExp].includes(varExp)))
+ this.variablesOccurrences[varExp].push(id);
+ }
+
+ /*
+ * Conducts the string match in query plan viewer
+ */
+ onClickSearch() {
+ if (this.searchRegex != "") {
+ this.searchMatches = [];
+
+ for (let searchId in this.flatJSONGraph) {
+ if (this.flatJSONGraph[searchId].includes(String(this.searchRegex).toLowerCase())) {
+ this.searchMatches.push(searchId);
+ }
+ }
+
+ if (this.searchMatches.length > 0) {
+ //matches found
+ this.matchesFound = true;
+ this.matchIdx = 0;
+
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ let nextIdx = this.ids.indexOf(this.searchMatches[this.matchIdx]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ }
+ }
+ }
+
+ /*
+ * Function that resets all the navigation variables
+ */
+ clearSelections() {
+ //clear main selection variables
+ this.nodeIdx = 0;
+ this.selectedNode = "n11";
+
+ //clear variable occurrences variables
+ this.selectedVariableOccurrences = undefined;
+ this.selectedVariableOccurrencesArray = [];
+ this.occurrenceArrayIdx = 0;
+
+ //clear search variables
+ this.searchRegex = "";
+ this.searchMatches = [];
+ this.matchesFound = false;
+ this.matchIdx = 0;
+
+ for (let node of this.nodesArr) {
+ node['selected'] = false;
+ }
+
+ this.panToNode(this.selectedNode);
+ }
+
+ /*
+ * Select variable from variable occurrences drop down
+ */
+ setSelectedVariableOccurrences(variable) {
+ this.selectedVariableOccurrences = variable;
+
+ //set the array
+ this.selectedVariableOccurrencesArray = this.variablesOccurrences[this.selectedVariableOccurrences];
+
+ //set the selected node to the first in the array
+ this.panToNode(this.selectedVariableOccurrencesArray[0]);
+ this.occurrenceArrayIdx = 0;
+ }
+
+ /*
+ * Jumps to selected variable's declaration (first occurrence in a DFS)
+ */
+ jumpToDeclaration() {
+ this.previousNodeId = this.selectedNode;
+ this.previouseOccurrenceArrayIdx = this.occurrenceArrayIdx;
+
+ this.panToNode(this.variablesDeclarations[this.selectedVariableOccurrences])
+
+ this.occurrenceArrayIdx = this.selectedVariableOccurrencesArray.length - 1;
+ }
+
+ /*
+ * Jump back to previous node after a call to jumpToDeclaration()
+ */
+ jumpBack() {
+ this.panToNode(this.previousNodeId);
+
+ this.occurrenceArrayIdx = this.previouseOccurrenceArrayIdx;
+
+ this.previousNodeId = undefined;
+ this.previouseOccurrenceArrayIdx = undefined;
+ }
+
+ /*
+ * Sets the orientation of the graph (top to bottom, left to right, etc.)
+ */
+ setOrientation(orientation: string): void {
+ this.planOrientation = orientation;
+ this.update$.next(true);
+ }
+
+ setDetail(checked: boolean) {
+ this.detailed = checked;
+ this.update$.next(true);
+ }
+
+ /*
+ * Pans to node in the graph. Jumps to specified node and updates selection variables
+ */
+ panToNode(id: any) {
+ this.selectedNode = id;
+
+ this.nodesArr[this.nodeIdx]['selected'] = false;
+
+ let nodeIdx = this.nodesArr.map(function(e) { return e.id}).indexOf(id);
+ this.nodeIdx = nodeIdx;
+ this.nodesArr[nodeIdx]['selected'] = true;
+
+ this.panToNode$.next(id);
+ this.update$.next(true);
+ }
+
+ /*
+ * Increments current node in graph (going "down" the graph in a DFS)
+ */
+ incrementNode() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ if (currentIdx + 1 < this.nodesArr.length) {
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[currentIdx + 1]['selected'] = true;
+ this.panToNode(this.ids[currentIdx + 1]);
+ this.nodeIdx = currentIdx + 1;
+ }
+ }
+
+ /*
+ * Decrements current node in graph (going "up" the graph in a DFS)
+ */
+ decrementNode() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+
+ if (currentIdx - 1 >= 0) {
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[currentIdx-1]['selected'] = true;
+ this.panToNode(this.ids[currentIdx - 1]);
+ this.nodeIdx = currentIdx - 1;
+ }
+ }
+
+ /*
+ * Increments current node but in occurrence (Jumping to the next occurrence of the selected variable)
+ */
+ incrementOccurrence() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ if (this.occurrenceArrayIdx + 1 < this.selectedVariableOccurrencesArray.length) {
+ let nextIdx = this.ids.indexOf(this.selectedVariableOccurrencesArray[this.occurrenceArrayIdx + 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.occurrenceArrayIdx += 1;
+ } else {
+ //wrap around to first item
+ let nextIdx = this.ids.indexOf(this.selectedVariableOccurrencesArray[0]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.occurrenceArrayIdx = 0;
+ }
+ }
+
+ /*
+ * Decrements current node but in occurrence (Jumping to the previous occurrence of the selected variable)
+ */
+ decrementOccurrence() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ if (this.occurrenceArrayIdx - 1 >= 0) {
+ let nextIdx = this.ids.indexOf(this.selectedVariableOccurrencesArray[this.occurrenceArrayIdx - 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.occurrenceArrayIdx -= 1;
+ } else {
+ //wrap around to last item
+ let nextIdx = this.ids.indexOf(this.selectedVariableOccurrencesArray[this.selectedVariableOccurrencesArray.length - 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.occurrenceArrayIdx = this.selectedVariableOccurrencesArray.length - 1;
+ }
+ }
+
+ /*
+ * Increments current node but in search match (Jumping to the next occurrence of the search results)
+ */
+ incrementMatch() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ if (this.matchIdx + 1 < this.searchMatches.length) {
+ let nextIdx = this.ids.indexOf(this.searchMatches[this.matchIdx + 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.matchIdx += 1;
+ } else {
+ //wrap around to first item
+ let nextIdx = this.ids.indexOf(this.searchMatches[0]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.matchIdx = 0;
+ }
+ }
+
+ /*
+ * Decrements current node but in search match (Jumping to the previous occurrence of the search results)
+ */
+ decrementMatch() {
+ let currentIdx = this.ids.indexOf(this.selectedNode);
+ if (this.matchIdx - 1 >= 0) {
+ let nextIdx = this.ids.indexOf(this.searchMatches[this.matchIdx - 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.matchIdx -= 1;
+ } else {
+ //wrap around to last item
+ let nextIdx = this.ids.indexOf(this.searchMatches[this.searchMatches.length - 1]);
+
+ this.nodesArr[currentIdx]['selected'] = false;
+ this.nodesArr[nextIdx]['selected'] = true;
+
+ this.panToNode(this.ids[nextIdx]);
+ this.nodeIdx = nextIdx;
+ this.matchIdx = this.searchMatches.length - 1;
+ }
+ }
+
+ /*
+ * Function takes in array of objects and stringifies
+ */
+ groupByListStringify(variables: any[]) {
+ let buildString = "";
+
+ let listSize = variables.length;
+ let counter = 0;
+
+ for (let variable of variables) {
+ if (counter < listSize - 1) {
+ buildString += variable['variable'] + " := " + variable['expression'] + "; ";
+ } else {
+ buildString += variable['variable'] + " := " + variable['expression'];
+ }
+
+ counter++;
+ }
+
+ return buildString;
+ }
+
+ /*
+ * Function that stringifys variables / objects array
+ */
+ variableExpressionStringify(arr: any[]) {
+ if (arr.length == 1) {
+ return "[" + arr[0] + "]";
+ } else {
+ return "[" + arr.toString() + "]";
+ }
+ }
+}
+
+
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.html
index a2e0d3e..ad40a2f 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.html
@@ -11,12 +11,14 @@
See the License for the specific language governing permissions and
limitations under the License.
*/ -->
+
<div class="query-container">
- <div class="content">
- <awc-query class="input-card"></awc-query>
- <awc-results class="output-card"></awc-results>
- </div>
- <div *ngIf="visible" class="drawer">
- <awc-metadata></awc-metadata>
- </div>
-</div>
\ No newline at end of file
+ <div class="content">
+ <awc-query (inputToOutputEmitter)="sendInputToOutput($event)" (isErrorEmitter)="sendIsError($event)" (hideOutputEmitter)="sendHideOutput($event)" class="input-card"></awc-query>
+ <awc-results [inputToOutput]="inputToOutput" [isError]="isError" [hideOutput]="hideOutput" class="output-card"></awc-results>
+ </div>
+ <div *ngIf="visible" class="drawer">
+ <awc-metadata></awc-metadata>
+ </div>
+</div>
+
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.scss
index 137721e..cdc901b 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.scss
@@ -33,7 +33,10 @@
}
.content {
+ display: flex;
+ flex-flow: column;
width: 100%;
+ overflow: auto;
}
.query-card {
@@ -52,4 +55,4 @@
width: 100%;
margin: 0;
padding: 0;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.ts
index aff5e50..236a5ae 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/query-container.component.ts
@@ -28,18 +28,29 @@
export class QueryContainerComponent implements AfterViewInit {
sideMenuVisible$: Observable<any>;
- visible = false;
+ visible = true;
+ inputToOutput: Object;
+ isError: boolean;
+ hideOutput: boolean;
constructor(private store: Store<any>) {}
ngAfterViewInit() {
this.sideMenuVisible$ = this.store.select(s => s.app.sideMenuVisible);
this.sideMenuVisible$.subscribe((data: any) => {
- if (data === true) {
- this.visible = true;
- } else {
- this.visible = false;
- }
+ this.visible = true;
})
}
-}
\ No newline at end of file
+
+ sendInputToOutput(inputToOutputData) {
+ this.inputToOutput = inputToOutputData;
+ }
+
+ sendIsError(isErrorData) {
+ this.isError = isErrorData;
+ }
+
+ sendHideOutput(hideOutputData) {
+ this.hideOutput = hideOutputData;
+ }
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
index b037a11..f6b0533 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.html
@@ -14,22 +14,23 @@
<div id='top' *ngIf='treeData' class='tree-card'>
<mat-expansion-panel hideToggle [expanded]="true">
<mat-expansion-panel-header class='header header-centered-v'>
- <mat-panel-title>OUTPUT DATA</mat-panel-title>
+ <mat-panel-title>QUERY OUTPUT</mat-panel-title>
<mat-icon>format_list_numbered_rtl</mat-icon>
</mat-expansion-panel-header>
<mat-panel-description class='content'>
<div class='panel'>
- <!--<span class='summary' >Items: {{metrics.resultCount}} Size: {{metrics.resultSizeKb}} Kb</span>-->
- <mat-paginator [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
- </mat-paginator>
<span class='options'>
- <button mat-button class='button' (click)='dataExpand()' [disabled]= 'checkView()' matTooltip="Expand Data"><mat-icon>add_circle</mat-icon></button>
- <button mat-button class='button' (click)='dataCollapse()' [disabled]= 'checkView()' matTooltip="Collapse Data"><mat-icon>remove_circle_outline</mat-icon></button>
- <button mat-button class='button button-json' (click)='showTable()' matTooltip="Show Table View">TABLE</button>
- <button mat-button class='button button-json' (click)='showTree()' matTooltip="Show Tree View">TREE</button>
- <button mat-button class='button button-json' (click)='showJSON()' matTooltip="Show JSON View">JSON</button>
- <button id='export' mat-button class='button' (click)='exportToText()' matTooltip="Export JSON file to Computer">EXPORT</button>
+ <button mat-button [ngClass]="this.tableVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showTable()' matTooltip="Show Table View">TABLE</button>
+ <button mat-button *ngIf="isCSV == false;" [ngClass]="this.jsonVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showJSON()' matTooltip="Show JSON View">JSON</button>
+ <button mat-button *ngIf="isCSV == false;" [ngClass]="this.treeVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showTree()' matTooltip="Show Tree View">TREE</button>
+
+ <button mat-button [ngClass]="this.planVisible === true ? 'button button-json selected' : 'button button-json'" (click)='showPlan()' matTooltip="Show Plan Viewer">PLAN</button>
+
+ <button *ngIf="isCSV" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export CSV file to Computer">EXPORT</button>
+ <button *ngIf="isCSV == false" mat-button class='button export' (click)='openJSONExportPicker()' matTooltip="Export JSON/JSONL file to Computer">EXPORT</button>
</span>
+ <mat-paginator *ngIf="this.planVisible === false" [showFirstLastButtons]="true" [length]='metrics.resultCount' [pageSize]='pagedefaults.pageSize' [pageSizeOptions]='pageSizeOptions' (page)='showResults($event, false)'>
+ </mat-paginator>
</div>
<div *ngIf='treeVisible' class='navi-data' class='navi-data'>
<mat-icon class='navi-path'>link</mat-icon>
@@ -37,7 +38,7 @@
</div>
<div class='divider'>
<div *ngIf='tableVisible'>
- <table mat-table [dataSource]="dataSource" class='items-table table-responsive'>
+ <table mat-table *ngIf='dataSource' [dataSource]="dataSource" class='items-table table-responsive'>
<ng-container matColumnDef="{{col}}" *ngFor="let col of displayedColumns">
<th mat-header-cell *matHeaderCellDef class='cell'>{{col}}</th>
<td mat-cell *matCellDef="let element"class='cell' >{{element[col]}}</td>
@@ -52,6 +53,9 @@
<div *ngIf='jsonVisible' class='json'>
<pre>{{jsonData}}</pre>
</div>
+ <div *ngIf="planVisible" class="plan">
+ <plan-viewer [planFormat]="planFormat" [jsonPlan]="jsonPlan" [plan]="plan" [planName]="planName"></plan-viewer>
+ </div>
<div id='bottom'></div>
</div>
<button *ngIf='showGoTop' mat-fab color='primary' class='button back-button' (click)='gotoTop()'>
@@ -59,4 +63,4 @@
</button>
</mat-panel-description>
</mat-expansion-panel>
-</div>
\ No newline at end of file
+</div>
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.scss b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.scss
index 5c2546f..6cc1283 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.scss
@@ -23,6 +23,7 @@
display: flex;
flex-flow: row;
align-items: center;
+ justify-content: space-between;
font-size: 12px;
padding-top: 15px;
border-bottom: 1px dashed gray;
@@ -53,6 +54,7 @@
border: 1px dashed gainsboro;
padding: 20px;
overflow: auto;
+ min-width: 0;
}
.content {
@@ -82,6 +84,10 @@
color: blue !important;
}
+.selected {
+ border: 2px solid blue !important;
+}
+
.navi-data {
display: flex;
flex-flow: row;
@@ -110,6 +116,10 @@
font-size: 14px;
}
+.export {
+ align-self: flex-end !important;
+}
+
.summary {
float: left;
margin-right: 15px;
@@ -169,4 +179,12 @@
overflow: hidden;
vertical-align: middle;
}
-}
\ No newline at end of file
+}
+
+.plan {
+ width: 100%;
+}
+
+.plan-new-tab {
+ margin-left: auto;
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
index f1437d9..9818973 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/dashboard/query/tree-view.component.ts
@@ -11,11 +11,17 @@
See the License for the specific language governing permissions and
limitations under the License.
*/
-import { Component, Input, NgZone, SimpleChange, ViewChild } from '@angular/core';
-import { MatTableDataSource } from '@angular/material/table';
+import { Component, Input, NgZone, SimpleChange, ViewChild, Inject } from '@angular/core';
+import {MatDialog, MatDialogRef, MAT_DIALOG_DATA} from '@angular/material/dialog';
+import {MatFooterCell, MatTableDataSource} from '@angular/material/table';
import { saveAs } from 'file-saver';
import * as cloneDeep from 'lodash/cloneDeep';
+export interface DialogData {
+ exportFormat: string;
+ fileName: string;
+}
+
@Component({
selector: 'tree-view',
templateUrl: 'tree-view.component.html',
@@ -25,10 +31,23 @@
export class TreeViewComponent {
@Input() data: any;
@Input() queryId: any;
+ @Input() planFormat: any;
+ @Input() plan: any;
+ @Input() planName: any;
+ @Input() jsonPlan: any;
+ @Input() inputToOutput: Object;
- jsonVisible: any = false;
- tableVisible: any = true;
+ isExplain: boolean = false;
+ outputFormat: string = 'json';
+ isCSV: boolean = false;
+ hasHeader: boolean = false;
+ exportFormat: string = 'json';
+ exportFileName: string = 'asterixdb-query-results';
+
+ jsonVisible: any = true;
+ tableVisible: any = false;
treeVisible: any = false;
+ planVisible: any = false;
jsonData: any;
jsonPath_: any = ': < JSON PATH >';
rawData: any;
@@ -54,7 +73,7 @@
private eventOptions: boolean|{capture?: boolean, passive?: boolean};
- constructor( private ngZone: NgZone) {}
+ constructor( private ngZone: NgZone, public dialog: MatDialog) {}
ngOnInit() {
this.ngZone.runOutsideAngular(() => {
@@ -63,9 +82,48 @@
}
ngOnChanges(changes: SimpleChange) {
- this.rawData = this.data['results'];
- if (this.rawData) {
- this.showResults(this.pagedefaults, this.EXPANDED);
+ if (this.inputToOutput) {
+ this.jsonVisible = true;
+ this.planVisible = false;
+ this.tableVisible = false;
+ this.treeVisible = false;
+ this.viewMode = 'JSON';
+
+ this.isExplain = this.inputToOutput['isExplain'];
+ this.outputFormat = this.inputToOutput['outputFormat'];
+
+ if (this.outputFormat == 'CSV_header') {
+ this.hasHeader = true;
+ this.outputFormat = 'CSV';
+ }
+
+ if (this.outputFormat == 'CSV' && !this.isExplain) {
+ this.exportFileName = 'asterixdb-query-results';
+ this.exportFormat = 'csv'
+ this.isCSV = true;
+ this.jsonVisible = false;
+ this.tableVisible = true;
+ this.viewMode = "TABLE";
+ }
+
+ if (this.data.length == 0) {
+ //case where 0 objects are returned
+ this.rawData = [];
+ this.showResults(this.pagedefaults, this.EXPANDED);
+ } else {
+ this.rawData = this.data['results'];
+ if (this.rawData) {
+ this.showResults(this.pagedefaults, this.EXPANDED);
+ }
+ }
+
+ if (this.isExplain) {
+ this.jsonVisible = false;
+ this.planVisible = true;
+ this.tableVisible = false;
+ this.treeVisible = false;
+ this.viewMode = "PLAN";
+ }
}
}
@@ -81,27 +139,48 @@
showResults(range, expanded) {
this.currentRange = range;
this.currentIndex = this.currentRange.pageIndex;
- this.treeData = this.rawData.filter(this.filter, this.currentRange);
- // Build the dynamic table column names
- // Flat the results to display in a table
- this.BuildTableFlatData(this.treeData);
- if (this.treeData.length > 0) {
+ if (this.rawData.length > 0) {
+ this.treeData = this.rawData.filter(this.filter, this.currentRange);
+ // Build the dynamic table column names
+ // Flat the results to display in a table
+ this.BuildTableFlatData(this.treeData);
+
+ if (this.treeData.length > 0) {
this.metrics = this.data['metrics'];
this.metrics['resultSizeKb'] = (this.metrics.resultSize/1024).toFixed(2);
var myData_ = [];
for (let i = 0; i < this.treeData.length; i++) {
- // mat-paginator start counting from 1, thats why the i+1 trick
- myData_.push(this.generateTree(this.treeData[i], '/', {}, (this.currentRange.pageSize * this.currentRange.pageIndex) + (i + 1), 0, expanded));
+ // mat-paginator start counting from 1, thats why the i+1 trick
+ myData_.push(this.generateTree(this.treeData[i], '/', {}, (this.currentRange.pageSize * this.currentRange.pageIndex) + (i + 1), 0, expanded));
}
this.treeData_ = myData_;
/* Prepare the JSON view */
this.jsonData = JSON.stringify(this.treeData, null, 8)
- } else {
+ } else {
console.log('no data')
this.treeData = [];
+ }
+ } else {
+ this.treeData = [];
+ this.jsonData = JSON.stringify([ ], null, 8);
+ this.metrics = {"resultCount": 0};
+
+ //clear tree data
+ var myData_ = [];
+ for (let i = 0; i < this.treeData.length; i++) {
+ // mat-paginator start counting from 1, thats why the i+1 trick
+ myData_.push(this.generateTree(this.treeData[i], '/', {}, (this.currentRange.pageSize * this.currentRange.pageIndex) + (i + 1), 0, expanded));
+ }
+
+ this.treeData_ = myData_;
+
+ //clear table data
+ this.dataSource.data = [];
+ this.displayedColumns = [];
}
+
}
/*
@@ -111,6 +190,8 @@
this.jsonVisible = true;
this.treeVisible = false;
this.tableVisible = false;
+ this.planVisible = false;
+ this.viewMode = 'JSON'
}
/*
@@ -120,6 +201,7 @@
this.jsonVisible = false;
this.treeVisible = false;
this.tableVisible = true;
+ this.planVisible = false;
this.viewMode = 'TABLE';
}
@@ -130,25 +212,63 @@
this.jsonVisible = false;
this.treeVisible = true;
this.tableVisible = false;
+ this.planVisible = false;
this.viewMode = 'TREE';
}
/*
+ * Shows Plan Viewer Mode
+ */
+ showPlan() {
+ this.jsonVisible = false;
+ this.treeVisible = false;
+ this.tableVisible = false;
+ this.planVisible = true;
+ this.viewMode = 'PLAN';
+ }
+
+ /*
* Export to CSV
*/
exportToCSV(){
- var exportOutput = JSON.stringify(this.rawData, null, 4);
- var blob = new Blob([this.jsonData], {type: "text/csv;charset=utf-8"});
- saveAs(blob, "Asterix-results.csv");
+ var csvJoin = this.rawData.join("");
+ var blob = new Blob([csvJoin], {type: "text/csv;charset=utf=8"});
+ if (this.exportFileName == "") {
+ saveAs(blob, "asterixdb-query-results.csv");
+ this.exportFileName = "asterixdb-query-results";
+ }
+ else {
+ saveAs(blob, this.exportFileName + ".csv");
+ }
}
/*
- * Export to plain text
+ * Export to JSON
*/
- exportToText(){
+ exportToJSON(){
var exportOutput = JSON.stringify(this.rawData, null, 4);
var blob = new Blob([exportOutput], {type: "text/json;charset=utf-8"});
- saveAs(blob, "Asterix-results.json");
+ if (this.exportFileName == "") {
+ saveAs(blob, "asterixdb-query-results.json");
+ this.exportFileName = "asterixdb-query-results";
+ }
+ else
+ saveAs(blob, this.exportFileName + ".json");
+ }
+
+ /*
+ * Export to JSON Lines (JSONL)
+ */
+ exportToJSONLines() {
+ var exportOutput = this.jsonlinesTransform(this.rawData);
+ var blob = new Blob([exportOutput], {type: "text/json;charset=utf-8"});
+ if (this.exportFileName == "") {
+ saveAs(blob, "asterixdb-query-results.jsonl");
+ this.exportFileName = "asterixdb-query-results";
+ }
+
+ else
+ saveAs(blob, this.exportFileName + ".jsonl");
}
/*
@@ -174,38 +294,42 @@
// Going through all the keys in a node looking for objects or array of key values
// and create a sub menu if is an object.
let nodeArray = [];
- Object.keys(node).map((k) => {
- if (typeof node[k] === 'object') {
- let nodeObject = { nested: true, item: '', label: '', key: '', value: '', type: '', link: '/', visible: expanded, children: [], level: level };
- nodeObject.item = index;
- nodeObject.label = k;
- nodeObject.key = k;
- nodeObject.value = node[k];
- nodeObject.link = nodeLink + '/' + k;
- nodeObject.level = level;
- level = level + 1;
- if(Array.isArray(node[k]) ){
- nodeObject.type = 'ARRAY';
- } else {
- nodeObject.type = 'OBJECT';
- }
- var newNodeObject = this.generateTree(node[k], nodeObject.link, nodeObject, index, level, expanded);
- if (nodeRoot.children) {
- nodeRoot.children.push(newNodeObject)
- }
+
+ if (node != null) {
+ Object.keys(node).map((k) => {
+ if (typeof node[k] === 'object' && (node[k] != null || node[k] != undefined)) {
+ let nodeObject = { nested: true, item: '', label: '', key: '', value: '', type: '', link: '/', visible: expanded, children: [], level: level };
+ nodeObject.item = index;
+ nodeObject.label = k;
+ nodeObject.key = k;
+ nodeObject.value = node[k];
+ nodeObject.link = nodeLink + '/' + k;
+ nodeObject.level = level;
+ level = level + 1;
+ if(Array.isArray(node[k]) ){
+ nodeObject.type = 'ARRAY';
+ } else {
+ nodeObject.type = 'OBJECT';
+ }
+ var newNodeObject = this.generateTree(node[k], nodeObject.link, nodeObject, index, level, expanded);
+ if (nodeRoot.children) {
+ nodeRoot.children.push(newNodeObject)
+ }
}
else {
- // key values converted into a unique string with a : separator
- let nodeKeyValue = { nested: false, item: '', label: '', key: '', value: '', type: 'KEYVALUE', link: '/', visible: expanded, children: [], level: level};
- nodeKeyValue.item = index;
- nodeKeyValue.label = k + " : " + node[k];
- nodeKeyValue.key = k;
- nodeKeyValue.value = node[k];
- nodeKeyValue.link = nodeLink + '/' + k + '/' + node[k];
- nodeKeyValue.level = level;
- nodeArray.push(nodeKeyValue);
+ // key values converted into a unique string with a : separator
+ let nodeKeyValue = { nested: false, item: '', label: '', key: '', value: '', type: 'KEYVALUE', link: '/', visible: expanded, children: [], level: level};
+ nodeKeyValue.item = index;
+ nodeKeyValue.label = k + " : " + node[k];
+ nodeKeyValue.key = k;
+ nodeKeyValue.value = node[k];
+ nodeKeyValue.link = nodeLink + '/' + k + '/' + node[k];
+ nodeKeyValue.level = level;
+ nodeArray.push(nodeKeyValue);
}
- })
+ })
+ }
+
// The array will be added as value to a parent key.
if (nodeArray.length > 0) {
nodeRoot.children = nodeArray.concat(nodeRoot.children)
@@ -258,55 +382,125 @@
this.jsonPath_ = event.link;
}
- dataExpand() {
- this.showResults(this.currentRange, this.EXPANDED);
- }
-
- dataCollapse() {
- this.showResults(this.currentRange, this.COLLAPSED);
- }
-
/*
* Build the table column names from result data
* Flat the result data for Table display
*/
BuildTableFlatData(data) {
+ if (this.isCSV)
+ this.buildTableFlatCSVData(data);
+ else
+ this.buildTableFlatJSONData(data);
+ }
- this.flattenData = [];
- this.displayedColumns = []
+ /*
+ * Reads JSON and creates data for table display
+ */
+ buildTableFlatJSONData(data) {
+ this.flattenData = [];
+ this.displayedColumns = []
- for (let i = 0; i < data.length; i++) {
- if (data[i] instanceof Object) {
+ const replacer = (key, value) => typeof value === 'undefined' ? null : value;
- var itemsKeyList = Object.keys(data[i]);
- var objectNode = cloneDeep(data[i]);
+ for (let i = 0; i < data.length; i++) {
+ if (data[i] instanceof Object) {
- for (let j = 0; j < itemsKeyList.length; j++) {
+ var itemsKeyList = Object.keys(data[i]);
+ var objectNode = cloneDeep(data[i]);
- var itemsKey: string = itemsKeyList[j];
- if (data[i][itemsKey] instanceof Object) {
- objectNode[itemsKey] = JSON.stringify(data[i][itemsKey], null, '\n');
- } else {
- objectNode[itemsKey] = data[i][itemsKey]
- }
- if (this.displayedColumns.indexOf(itemsKey) === -1) {
- this.displayedColumns.push(itemsKey)
- }
- }
- this.flattenData.push(objectNode)
+ for (let j = 0; j < itemsKeyList.length; j++) {
+
+ var itemsKey: string = itemsKeyList[j];
+ if (data[i][itemsKey] instanceof Object) {
+ objectNode[itemsKey] = JSON.stringify(data[i][itemsKey], replacer, '\n');
} else {
- this.displayedColumns.push('value')
- this.flattenData.push({ 'value': data[0] })
+ objectNode[itemsKey] = data[i][itemsKey]
}
+ if (this.displayedColumns.indexOf(itemsKey) === -1) {
+ this.displayedColumns.push(itemsKey)
+ }
+ }
+ this.flattenData.push(objectNode)
+ } else {
+ //only create one column called "value" if you are doing a select value
+ if (i == 0)
+ this.displayedColumns.push('value')
+ this.flattenData.push({ 'value': data[i] })
+ }
+ }
+
+ this.dataSource.data = this.flattenData;
+ }
+
+ /*
+ * Reads CSV data and creates data for table display
+ */
+ buildTableFlatCSVData(data) {
+ this.flattenData = [];
+ this.displayedColumns = [];
+
+ for (let i = 0; i < data.length; i++) {
+ //split the string using the comma delimiter
+ let items = this.readCSV(data[i]);
+
+ //iterate over these elements
+ let item_idx = 0;
+ let row_obj = {};
+
+ for (let item of items) {
+ //if it is the header row
+ if (this.hasHeader && i == 0) {
+ this.displayedColumns.push(item);
+ } else if (!this.hasHeader && i == 0) {
+ this.displayedColumns.push("Column " + (item_idx + 1).toString());
+ row_obj["Column " + (item_idx + 1).toString()] = item;
+ } else {
+ if (this.displayedColumns.length > 0) {
+ //has header
+ row_obj[this.displayedColumns[item_idx]] = item;
+ } else {
+ //does not have header
+ row_obj["Column " + (item_idx + 1).toString()] = item;
+ }
+ }
+
+ item_idx++;
}
- this.dataSource.data = this.flattenData;
+ if (Object.keys(row_obj).length > 0)
+ this.flattenData.push(row_obj);
+ }
+
+ this.dataSource.data = this.flattenData;
}
jsonTransform(item) {
return JSON.stringify(item, null, 4);
}
+ /*
+ * function transformers json item into a string of JSONL
+ */
+ jsonlinesTransform(item) {
+ let buildString = "";
+
+ let counter = 0;
+
+ let newLineRE = /\r?\n|\r/g;
+
+ for (let obj of item) {
+ buildString += JSON.stringify(obj).replace(newLineRE, "");
+
+ //new line delimeter
+ if (counter < item.length - 1)
+ buildString += "\n";
+
+ counter++;
+ }
+
+ return buildString;
+ }
+
checkView() {
if (!this.treeVisible) {
return true;
@@ -314,4 +508,76 @@
return false
}
}
-}
\ No newline at end of file
+
+ /*
+ * Function opens dialog to pick between JSON and JSONL
+ */
+ openJSONExportPicker() {
+ const dialogRef = this.dialog.open(DialogExportPicker, {
+ width: '350px',
+ data: {exportFormat: this.exportFormat, fileName: this.exportFileName}
+ });
+
+ dialogRef.afterClosed().subscribe(result => {
+ if (result[0] != 'cancel') {
+ this.exportFormat = result[0];
+ this.exportFileName = result[1];
+
+ if (this.exportFormat == 'json')
+ this.exportToJSON();
+ else if (this.exportFormat == 'jsonl')
+ this.exportToJSONLines();
+ else if (this.exportFormat == 'csv')
+ this.exportToCSV();
+
+ if (this.exportFormat == 'jsonl')
+ this.exportFormat = 'json';
+ }
+ });
+ }
+
+ /*
+ * Function reads row of CSV and returns the list of items
+ */
+ readCSV(row: string) {
+ let items = []
+ let escapeMode = false;
+ let currentItem = "";
+
+ for (let i = 0; i < row.length; i++) {
+ let curr_char = row.charAt(i);
+
+ if (curr_char == '"' && escapeMode)
+ //turn off escape mode
+ escapeMode = false;
+ else if (curr_char == '"' && !escapeMode)
+ //turn on escape mode
+ escapeMode = true;
+ else {
+ //if char is a comma and not in escape mode
+ if ((curr_char == ',' && !escapeMode) || i == row.length - 1) {
+ //push current item and reset for next item
+ items.push(currentItem);
+ currentItem = "";
+ } else {
+ currentItem += curr_char;
+ }
+ }
+ }
+
+ return items;
+ }
+}
+
+@Component({
+ selector: 'dialog-export-picker',
+ templateUrl: 'dialog-export-picker.html',
+ styleUrls: ['dialog-export-picker.scss']
+})
+export class DialogExportPicker {
+
+ constructor(
+ public dialogRef: MatDialogRef<DialogExportPicker>,
+ @Inject(MAT_DIALOG_DATA) public data: DialogData) {}
+}
+
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/material.module.ts b/asterixdb/asterix-dashboard/src/node/src/app/material.module.ts
index d172bf8..756d758 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/material.module.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/material.module.ts
@@ -11,40 +11,39 @@
See the License for the specific language governing permissions and
limitations under the License.
*/
-import {NgModule} from '@angular/core';
-import {
- MatAutocompleteModule,
- MatButtonModule,
- MatButtonToggleModule,
- MatCardModule,
- MatCheckboxModule,
- MatChipsModule,
- MatDatepickerModule,
- MatDialogModule,
- MatExpansionModule,
- MatFormFieldModule,
- MatGridListModule,
- MatIconModule,
- MatInputModule,
- MatListModule,
- MatMenuModule,
- MatPaginatorModule,
- MatProgressBarModule,
- MatProgressSpinnerModule,
- MatRadioModule,
- MatSelectModule,
- MatSidenavModule,
- MatSliderModule,
- MatSlideToggleModule,
- MatSnackBarModule,
- MatSortModule,
- MatTableModule,
- MatTabsModule,
- MatToolbarModule,
- MatTooltipModule,
- MatStepperModule,
-} from '@angular/material';
-import {MatNativeDateModule, MatRippleModule} from '@angular/material';
+import {NgModule, CUSTOM_ELEMENTS_SCHEMA} from '@angular/core';
+import { MatAutocompleteModule } from '@angular/material/autocomplete';
+import { MatButtonModule } from "@angular/material/button";
+import { MatButtonToggleModule } from "@angular/material/button-toggle";
+import { MatCardModule } from "@angular/material/card";
+import { MatCheckboxModule } from "@angular/material/checkbox";
+import { MatChipsModule } from "@angular/material/chips";
+import { MatDatepickerModule } from "@angular/material/datepicker";
+import { MatDialogModule } from "@angular/material/dialog";
+import { MatExpansionModule } from "@angular/material/expansion";
+import { MatFormFieldModule } from "@angular/material/form-field";
+import { MatGridListModule } from "@angular/material/grid-list";
+import { MatIconModule } from "@angular/material/icon";
+import { MatInputModule } from "@angular/material/input";
+import { MatListModule } from "@angular/material/list";
+import { MatMenuModule } from "@angular/material/menu";
+import { MatPaginatorModule } from "@angular/material/paginator";
+import { MatProgressBarModule } from "@angular/material/progress-bar";
+import { MatProgressSpinnerModule } from "@angular/material/progress-spinner";
+import { MatRadioModule } from "@angular/material/radio";
+import { MatSelectModule } from "@angular/material/select";
+import { MatSidenavModule } from "@angular/material/sidenav";
+import { MatSliderModule } from "@angular/material/slider";
+import { MatSlideToggleModule } from "@angular/material/slide-toggle";
+import { MatSnackBarModule } from "@angular/material/snack-bar";
+import { MatSortModule } from "@angular/material/sort";
+import { MatTableModule } from "@angular/material/table";
+import { MatTabsModule } from "@angular/material/tabs";
+import { MatToolbarModule } from "@angular/material/toolbar";
+import { MatTooltipModule } from "@angular/material/tooltip";
+import { MatTreeModule } from "@angular/material/tree";
+import {MatStepperModule } from "@angular/material/stepper";
+import { MatNativeDateModule, MatRippleModule } from "@angular/material/core";
import {CdkTableModule} from '@angular/cdk/table';
//import {CdkAccordionModule} from '@angular/cdk/accordion';
import {A11yModule} from '@angular/cdk/a11y';
@@ -53,6 +52,8 @@
import {PlatformModule} from '@angular/cdk/platform';
import {ObserversModule} from '@angular/cdk/observers';
import {PortalModule} from '@angular/cdk/portal';
+import {DragDropModule} from "@angular/cdk/drag-drop";
+import {MatBadgeModule} from "@angular/material/badge";
/*
* NgModule that includes all Material modules that are required to
@@ -92,6 +93,7 @@
MatTableModule,
MatToolbarModule,
MatTooltipModule,
+ MatTreeModule,
MatNativeDateModule,
CdkTableModule,
A11yModule,
@@ -101,6 +103,9 @@
OverlayModule,
PlatformModule,
PortalModule,
- ]
+ DragDropModule,
+ MatBadgeModule,
+ ],
+ schemas: [CUSTOM_ELEMENTS_SCHEMA]
})
-export class MaterialModule {}
\ No newline at end of file
+export class MaterialModule {}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/cancel.actions.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/cancel.actions.ts
new file mode 100644
index 0000000..a38e106
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/cancel.actions.ts
@@ -0,0 +1,43 @@
+/*
+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.
+*/
+import { Action } from '@ngrx/store';
+
+export const CANCEL_QUERY = '[Query] Cancel SQL++ Query';
+export const CANCEL_QUERY_SUCCESS = '[Query] Cancel SQL++ Query Success';
+export const CANCEL_QUERY_FAIL = '[Query] Cancel SQL++ Query Fail';
+
+/*
+* Cancel SQL++ Query
+ */
+export class CancelQuery implements Action {
+ readonly type = CANCEL_QUERY;
+ constructor(public payload: any) {}
+}
+
+export class CancelQuerySuccess implements Action {
+ readonly type = CANCEL_QUERY_SUCCESS;
+ constructor(public payload: any) {}
+}
+
+export class CancelQueryFail implements Action {
+ readonly type = CANCEL_QUERY_FAIL;
+ constructor(public payload: any) {}
+}
+
+/*
+* Exports of SQL++ actions
+*/
+export type All = CancelQuery |
+ CancelQuerySuccess |
+ CancelQueryFail;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/dataset.actions.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/dataset.actions.ts
index 61259bb..40214d9 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/dataset.actions.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/dataset.actions.ts
@@ -29,7 +29,9 @@
export const DROP_DATASET_SUCCESS = '[Dataset Collection] Drop Dataset Success';
export const DROP_DATASET_FAIL = '[Dataset Collection] Drop Dataset Fail';
export const GUIDE_SELECT_DATASET = '[Dataset Collection] Guide Select Dataset';
-
+export const SAMPLE_DATASET = '[Dataset Collection] Sample Dataset';
+export const SAMPLE_DATASET_SUCCESS = '[Dataset Collection] Sample Dataset Success';
+export const SAMPLE_DATASET_FAIL = '[Dataset Collection] Sample Dataset Fail';
/*
* Guide Select Datasets for UI Helpers
@@ -112,6 +114,24 @@
}
/*
+* Sample Dataset
+ */
+export class SampleDataset implements Action {
+ readonly type = SAMPLE_DATASET;
+ constructor(public payload: any) {}
+}
+
+export class SampleDatasetSuccess implements Action {
+ readonly type = SAMPLE_DATASET_SUCCESS;
+ constructor(public payload: any) {}
+}
+
+export class SampleDatasetFail implements Action {
+ readonly type = SAMPLE_DATASET_FAIL;
+ constructor(public payload: any) {}
+}
+
+/*
* Exports of datasets actions
*/
export type All = SelectDatasets |
@@ -126,4 +146,7 @@
DropDataset |
DropDatasetSuccess |
DropDatasetFail |
- GuideSelectDatasets;
\ No newline at end of file
+ SampleDataset |
+ SampleDatasetSuccess |
+ SampleDatasetFail |
+ GuideSelectDatasets;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/function.actions.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/function.actions.ts
new file mode 100644
index 0000000..0f10153
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/function.actions.ts
@@ -0,0 +1,46 @@
+/*
+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.
+*/
+import { Action } from '@ngrx/store';
+
+/*
+* Definition of Function Actions
+*/
+export const SELECT_FUNCTIONS = '[Function Collection] Select Functions';
+export const SELECT_FUNCTIONS_SUCCESS = '[Function Collection] Select Functions Success';
+export const SELECT_FUNCTIONS_FAIL = '[Function Collection] Select Functions Fail';
+
+/*
+* Select Functions
+*/
+export class SelectFunctions implements Action {
+ readonly type = SELECT_FUNCTIONS;
+ constructor(public payload: string) {}
+}
+
+export class SelectFunctionsSuccess implements Action {
+ readonly type = SELECT_FUNCTIONS_SUCCESS;
+ constructor(public payload: any[]) {}
+}
+
+export class SelectFunctionsFail implements Action {
+ readonly type = SELECT_FUNCTIONS_FAIL;
+ constructor(public payload: any[]) {}
+}
+
+/*
+* Exports of functions actions
+*/
+export type All = SelectFunctions |
+ SelectFunctionsSuccess |
+ SelectFunctionsFail;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/query.actions.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/query.actions.ts
index e18a89e..59366f6 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/query.actions.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/actions/query.actions.ts
@@ -87,4 +87,4 @@
ExecuteQueryFail |
ExecuteMetadataQuery |
ExecuteMetadataQuerySuccess |
- ExecuteMetadataQueryFail;
\ No newline at end of file
+ ExecuteMetadataQueryFail;
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/app.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/app.effects.ts
index 891191b..80864e8 100644
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/app.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/app.effects.ts
@@ -16,9 +16,9 @@
import { Actions } from '@ngrx/effects';
import * as appActions from '../actions/app.actions';
-export type Action = appActions.All
+export type Action_type = appActions.All
@Injectable()
export class AppEffects {
constructor(private actions: Actions) {}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/cancel.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/cancel.effects.ts
new file mode 100644
index 0000000..a025c8b
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/cancel.effects.ts
@@ -0,0 +1,43 @@
+/*
+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.
+*/
+import { Injectable } from '@angular/core';
+import { Action } from '@ngrx/store';
+import { Actions, Effect, ofType } from '@ngrx/effects';
+import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from 'rxjs/operators';
+import { SQLService } from '../services/async-query.service';
+import * as sqlCancelActions from '../actions/cancel.actions';
+
+export type Action_type = sqlCancelActions.All;
+
+@Injectable()
+export class SQLCancelEffects {
+ constructor(private actions: Actions,
+ private sqlService: SQLService) {
+ }
+
+ /*
+ * Effect to Cancel a SQL++ Query against AsterixDB
+ */
+ @Effect()
+ cancelQuery$: Observable<Action_type> = this.actions.pipe(
+ ofType(sqlCancelActions.CANCEL_QUERY),
+ switchMap(query => {
+ return this.sqlService.cancelSQLQuery((query as any).payload.requestId).pipe(
+ map(sqlCancelQueryResult => new sqlCancelActions.CancelQuerySuccess(sqlCancelQueryResult)),
+ catchError(sqlCancelQueryError => of(new sqlCancelActions.CancelQueryFail(sqlCancelQueryError)))
+ )
+ })
+ )
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataset.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataset.effects.ts
index 3ca0da4..0a5bb5a 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataset.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataset.effects.ts
@@ -13,13 +13,13 @@
*/
import { Injectable } from '@angular/core';
import { Action } from '@ngrx/store';
-import { Effect, Actions } from '@ngrx/effects';
+import { Effect, Actions, ofType} from '@ngrx/effects';
import { Observable , of } from 'rxjs';
+import { switchMap, map, catchError } from "rxjs/operators";
import * as datasetActions from '../actions/dataset.actions';
import { SQLService } from '../services/async-query.service';
-import 'rxjs/add/operator/switchMap'
-export type Action = datasetActions.All
+export type ActionType = datasetActions.All
@Injectable()
export class DatasetEffects {
@@ -29,33 +29,52 @@
/* Effect to load a collection of all Datasets from AsterixDB
*/
@Effect()
- selectDatasets$: Observable<Action> = this.actions
- .ofType(datasetActions.SELECT_DATASETS)
- .switchMap(query => {
- return this.sqlService.selectDatasets()
- .map(dataset => new datasetActions.SelectDatasetsSuccess(dataset))
- .catch(err => of(new datasetActions.SelectDatasetsFail(err)));
- });
+ selectDatasets$: Observable<ActionType> = this.actions.pipe(
+ ofType(datasetActions.SELECT_DATASETS),
+ switchMap(query => {
+ return this.sqlService.selectDatasets().pipe(
+ map(dataset => new datasetActions.SelectDatasetsSuccess(dataset)),
+ catchError(err => of(new datasetActions.SelectDatasetsFail(err)))
+ )
+ })
+ );
/* Effect to create a Datasets from AsterixDB
*/
@Effect()
- createDatasets$: Observable<Action> = this.actions
- .ofType(datasetActions.CREATE_DATASET)
- .switchMap(dataset => {
- return this.sqlService.createDataset((dataset as any).payload)
- .map(dataset => new datasetActions.CreateDatasetSuccess(dataset))
- .catch(err => of(new datasetActions.CreateDatasetFail(err)));
- });
+ createDatasets$: Observable<ActionType> = this.actions.pipe(
+ ofType(datasetActions.CREATE_DATASET),
+ switchMap(dataset => {
+ return this.sqlService.createDataset((dataset as any).payload).pipe(
+ map(dataset => new datasetActions.CreateDatasetSuccess(dataset)),
+ catchError(err => of(new datasetActions.CreateDatasetFail(err)))
+ )
+ })
+ );
/* Effect to drop a Datasets from AsterixDB
*/
@Effect()
- dropDatasets$: Observable<Action> = this.actions
- .ofType(datasetActions.DROP_DATASET)
- .switchMap(dataset => {
- return this.sqlService.dropDataset((dataset as any).payload)
- .map(dataset => new datasetActions.DropDatasetSuccess(dataset))
- .catch(err => of(new datasetActions.DropDatasetFail(err)));
- });
-}
\ No newline at end of file
+ dropDatasets$: Observable<ActionType> = this.actions.pipe(
+ ofType(datasetActions.DROP_DATASET),
+ switchMap(dataset => {
+ return this.sqlService.dropDataset((dataset as any).payload).pipe(
+ map(dataset => new datasetActions.DropDatasetSuccess(dataset)),
+ catchError(err => of(new datasetActions.DropDatasetFail(err)))
+ )
+ })
+ );
+
+ /* Effect of sampling a datasets from AsterixDB
+ */
+ @Effect()
+ sampleDataset$: Observable<ActionType> = this.actions.pipe(
+ ofType(datasetActions.SAMPLE_DATASET),
+ switchMap(dataset => {
+ return this.sqlService.sampleDataset((dataset as any).payload.dataset).pipe(
+ map(dataset => new datasetActions.SampleDatasetSuccess(dataset)),
+ catchError(err => of(new datasetActions.SampleDatasetFail(err)))
+ )
+ })
+ );
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/datatype.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/datatype.effects.ts
index 5e06f89..50daa1b 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/datatype.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/datatype.effects.ts
@@ -12,8 +12,9 @@
limitations under the License.
*/
import { Injectable } from '@angular/core';
-import { Effect, Actions } from '@ngrx/effects';
+import { Effect, Actions, ofType } from '@ngrx/effects';
import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from "rxjs/operators";
import * as datatypeActions from '../actions/datatype.actions';
import { SQLService } from '../services/async-query.service';
@@ -26,33 +27,38 @@
/* Effect to load a collection of all Datatypes from AsterixDB */
@Effect()
- selectDatatypes$: Observable<Action> = this.actions
- .ofType(datatypeActions.SELECT_DATATYPES)
- .switchMap(query => {
- return this.sqlService.selectDatatypes()
- .map(datatype => new datatypeActions.SelectDatatypesSuccess(datatype))
- .catch(err => of(new datatypeActions.SelectDatatypesFail(err)));
- });
+ selectDatatypes$: Observable<Action> = this.actions.pipe(
+ ofType(datatypeActions.SELECT_DATATYPES),
+ switchMap(query => {
+ return this.sqlService.selectDatatypes().pipe(
+ map(datatype => new datatypeActions.SelectDatatypesSuccess(datatype)),
+ catchError(err => of(new datatypeActions.SelectDatatypesFail(err)))
+ )
+ })
+ );
/* Effect to create a Datatype from AsterixDB
*/
@Effect()
- createDatatypes$: Observable<Action> = this.actions
- .ofType(datatypeActions.CREATE_DATATYPE)
- .switchMap(datatype => {
- return this.sqlService.createDatatype((datatype as any).payload)
- .map(datatype => new datatypeActions.CreateDatatypeSuccess(datatype))
- .catch(err => of(new datatypeActions.CreateDatatypeFail(err)));
- });
-
+ createDatatypes$: Observable<Action> = this.actions.pipe(
+ ofType(datatypeActions.CREATE_DATATYPE),
+ switchMap(datatype => {
+ return this.sqlService.createDatatype((datatype as any).payload).pipe(
+ map(datatype => new datatypeActions.CreateDatatypeSuccess(datatype)),
+ catchError(err => of(new datatypeActions.CreateDatatypeFail(err)))
+ )
+ })
+ );
/* Effect to drop a Datatype from AsterixDB
*/
@Effect()
- dropDatatypes$: Observable<Action> = this.actions
- .ofType(datatypeActions.DROP_DATATYPE)
- .switchMap(datatype => {
- return this.sqlService.dropDatatype((datatype as any).payload)
- .map(datatype => new datatypeActions.DropDatatypeSuccess(datatype))
- .catch(err => of(new datatypeActions.DropDatatypeFail(err)));
- });
-}
\ No newline at end of file
+ dropDatatypes$: Observable<Action> = this.actions.pipe(
+ ofType(datatypeActions.DROP_DATATYPE),
+ switchMap(datatype => {
+ return this.sqlService.dropDatatype((datatype as any).payload).pipe(
+ map(datatype => new datatypeActions.DropDatatypeSuccess(datatype)),
+ catchError(err => of(new datatypeActions.DropDatatypeFail(err)))
+ )
+ })
+ );
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataverse.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataverse.effects.ts
index e8e86af..a37f4d2 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataverse.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/dataverse.effects.ts
@@ -12,8 +12,9 @@
limitations under the License.
*/
import { Injectable } from '@angular/core';
-import { Effect, Actions } from '@ngrx/effects';
+import { Effect, Actions, ofType } from '@ngrx/effects';
import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from "rxjs/operators";
import * as dataverseActions from '../actions/dataverse.actions';
import { SQLService } from '../services/async-query.service';
@@ -22,44 +23,50 @@
@Injectable()
export class DataverseEffects {
constructor(private actions: Actions, private sqlService: SQLService) {}
-
/* Effect to set the default Dataverse */
@Effect()
- setDefaultDataverse$: Observable<Action> = this.actions
- .ofType(dataverseActions.SET_DEFAULT_DATAVERSE)
- .switchMap(query => {
- return new Observable().map(dataverse => new dataverseActions.SetDefaultDataverse('Default'))
- });
+ setDefaultDataverse$: Observable<Action> = this.actions.pipe(
+ ofType(dataverseActions.SET_DEFAULT_DATAVERSE),
+ switchMap(query => {
+ return new Observable().pipe(map(dataverse => new dataverseActions.SetDefaultDataverse('Default')))
+ })
+ );
/* Effect to load a collection of all Dataverses from AsterixDB */
@Effect()
- selectDataverses$: Observable<Action> = this.actions
- .ofType(dataverseActions.SELECT_DATAVERSES)
- .switchMap(query => {
- return this.sqlService.selectDataverses()
- .map(dataverse => new dataverseActions.SelectDataversesSuccess(dataverse))
- .catch(err => of(new dataverseActions.SelectDataversesFail(err)));
- });
+ selectDataverses$: Observable<Action> = this.actions.pipe(
+ ofType(dataverseActions.SELECT_DATAVERSES),
+ switchMap(query => {
+ return this.sqlService.selectDataverses().pipe(
+ map(dataverse => new dataverseActions.SelectDataversesSuccess(dataverse)),
+ catchError(err => of(new dataverseActions.SelectDataversesFail(err)))
+ )
+ })
+ );
/* Effect to create Dataverse from AsterixDB
*/
@Effect()
- createDataverses$: Observable<Action> = this.actions
- .ofType(dataverseActions.CREATE_DATAVERSE)
- .switchMap(dataverseName => {
- return this.sqlService.createDataverse((dataverseName as any).payload)
- .map(dataverse => new dataverseActions.CreateDataverseSuccess(dataverse))
- .catch(err => of(new dataverseActions.CreateDataverseFail(err)));
- });
+ createDataverses$: Observable<Action> = this.actions.pipe(
+ ofType(dataverseActions.CREATE_DATAVERSE),
+ switchMap(dataverseName => {
+ return this.sqlService.createDataverse((dataverseName as any).payload).pipe(
+ map(dataverse => new dataverseActions.CreateDataverseSuccess(dataverse)),
+ catchError(err => of(new dataverseActions.CreateDataverseFail(err)))
+ )
+ })
+ );
/* Effect to drop a Dataverse from AsterixDB
*/
@Effect()
- dropDataverses$: Observable<Action> = this.actions
- .ofType(dataverseActions.DROP_DATAVERSE)
- .switchMap(dataverseName => {
- return this.sqlService.dropDataverse((dataverseName as any).payload)
- .map(dataverse => new dataverseActions.DropDataverseSuccess(dataverse))
- .catch(err => of(new dataverseActions.DropDataverseFail(err)));
- });
+ dropDataverses$: Observable<Action> = this.actions.pipe(
+ ofType(dataverseActions.DROP_DATAVERSE),
+ switchMap(dataverseName => {
+ return this.sqlService.dropDataverse((dataverseName as any).payload).pipe(
+ map(dataverse => new dataverseActions.DropDataverseSuccess(dataverse)),
+ catchError(err => of(new dataverseActions.DropDataverseFail(err)))
+ )
+ })
+ );
}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/function.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/function.effects.ts
new file mode 100644
index 0000000..de0a102
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/function.effects.ts
@@ -0,0 +1,40 @@
+/*
+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.
+*/
+
+import { Injectable } from '@angular/core';
+import { Effect, Actions, ofType } from '@ngrx/effects';
+import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from 'rxjs/operators';
+import * as functionActions from '../actions/function.actions';
+import { SQLService } from "../services/async-query.service";
+
+export type Action = functionActions.All;
+
+@Injectable()
+export class FunctionEffects {
+ constructor(private actions: Actions,
+ private sqlService: SQLService) {}
+
+ /* Effect to load a collection of all functions from AsterixDB */
+ @Effect()
+ selectFunctions$: Observable<Action> = this.actions.pipe(
+ ofType(functionActions.SELECT_FUNCTIONS),
+ switchMap(query => {
+ return this.sqlService.selectFunctions().pipe(
+ map(fn => new functionActions.SelectFunctionsSuccess(fn)),
+ catchError(err => of(new functionActions.SelectFunctionsFail(err)))
+ )
+ })
+ );
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/index.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/index.effects.ts
index a1c0b07..7a70a06 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/index.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/index.effects.ts
@@ -12,8 +12,9 @@
limitations under the License.
*/
import { Injectable } from '@angular/core';
-import { Effect, Actions } from '@ngrx/effects';
+import { Effect, Actions, ofType } from '@ngrx/effects';
import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from 'rxjs/operators';
import * as indexActions from '../actions/index.actions';
import { SQLService } from '../services/async-query.service';
@@ -26,33 +27,39 @@
/* Effect to load a collection of all Index from AsterixDB */
@Effect()
- selectIndexes$: Observable<Action> = this.actions
- .ofType(indexActions.SELECT_INDEXES)
- .switchMap(query => {
- return this.sqlService.selectIndexes()
- .map(index => new indexActions.SelectIndexesSuccess(index))
- .catch(err => of(new indexActions.SelectIndexesFail(err)));
- });
+ selectIndexes$: Observable<Action> = this.actions.pipe(
+ ofType(indexActions.SELECT_INDEXES),
+ switchMap(query => {
+ return this.sqlService.selectIndexes().pipe(
+ map(index => new indexActions.SelectIndexesSuccess(index)),
+ catchError(err => of(new indexActions.SelectIndexesFail(err)))
+ )
+ })
+ );
/* Effect to create a Index
*/
@Effect()
- createIndexes$: Observable<Action> = this.actions
- .ofType(indexActions.CREATE_INDEX)
- .switchMap(index => {
- return this.sqlService.createIndex((index as any).payload)
- .map(index => new indexActions.CreateIndexSuccess(index))
- .catch(err => of(new indexActions.CreateIndexFail(err)));
- });
+ createIndexes$: Observable<Action> = this.actions.pipe(
+ ofType(indexActions.CREATE_INDEX),
+ switchMap(index => {
+ return this.sqlService.createIndex((index as any).payload).pipe(
+ map(index => new indexActions.CreateIndexSuccess(index)),
+ catchError(err => of(new indexActions.CreateIndexFail(err)))
+ )
+ })
+ );
/* Effect to drop a Index
*/
@Effect()
- dropIndexes$: Observable<Action> = this.actions
- .ofType(indexActions.DROP_INDEX)
- .switchMap(index => {
- return this.sqlService.dropIndex((index as any).payload)
- .map(index => new indexActions.DropIndexSuccess(index))
- .catch(err => of(new indexActions.DropIndexFail(err)));
- });
+ dropIndexes$: Observable<Action> = this.actions.pipe(
+ ofType(indexActions.DROP_INDEX),
+ switchMap(index => {
+ return this.sqlService.dropIndex((index as any).payload).pipe(
+ map(index => new indexActions.DropIndexSuccess(index)),
+ catchError(err => of(new indexActions.DropIndexFail(err)))
+ )
+ })
+ );
}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/query.effects.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/query.effects.ts
index 317211f..a5f8127 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/query.effects.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/effects/query.effects.ts
@@ -15,10 +15,11 @@
import { Action } from '@ngrx/store';
import { Actions, Effect, ofType } from '@ngrx/effects';
import { Observable , of } from 'rxjs';
+import { map, switchMap, catchError } from 'rxjs/operators';
import { SQLService } from '../services/async-query.service';
import * as sqlQueryActions from '../actions/query.actions';
-export type Action = sqlQueryActions.All
+export type Action_type = sqlQueryActions.All
@Injectable()
export class SQLQueryEffects {
@@ -27,22 +28,26 @@
/* Effect to Execute an SQL++ Query against the AsterixDB */
@Effect()
- executeQuery$: Observable<Action> = this.actions
- .ofType(sqlQueryActions.EXECUTE_QUERY)
- .switchMap(query => {
- return this.sqlService.executeSQLQuery((query as any).payload.queryString, (query as any).payload.planFormat)
- .map(sqlQueryResult => new sqlQueryActions.ExecuteQuerySuccess(sqlQueryResult))
- .catch(sqlQueryError => of(new sqlQueryActions.ExecuteQueryFail(sqlQueryError)));
- });
+ executeQuery$: Observable<Action_type> = this.actions.pipe(
+ ofType(sqlQueryActions.EXECUTE_QUERY),
+ switchMap(query => {
+ return this.sqlService.executeSQLQuery((query as any).payload.queryString, (query as any).payload.planFormat, (query as any).payload.format, (query as any).payload.requestId).pipe(
+ map(sqlQueryResult => new sqlQueryActions.ExecuteQuerySuccess(sqlQueryResult)),
+ catchError(sqlQueryError => of(new sqlQueryActions.ExecuteQueryFail(sqlQueryError)))
+ )
+ })
+ );
/* Effect to Execute an SQL++ Metadata Query against the AsterixDB
*/
@Effect()
- executeMetadataQuery$: Observable<Action> = this.actions
- .ofType(sqlQueryActions.EXECUTE_METADATA_QUERY)
- .switchMap(query => {
- return this.sqlService.executeSQLQuery((query as any).payload, (query as any).payload.planFormat)
- .map(sqlMetadataQueryResult => new sqlQueryActions.ExecuteMetadataQuerySuccess(sqlMetadataQueryResult))
- .catch(sqlMetadataQueryError => of(new sqlQueryActions.ExecuteMetadataQueryFail(sqlMetadataQueryError)));
- });
-}
\ No newline at end of file
+ executeMetadataQuery$: Observable<Action_type> = this.actions.pipe(
+ ofType(sqlQueryActions.EXECUTE_METADATA_QUERY),
+ switchMap(query => {
+ return this.sqlService.executeSQLQuery((query as any).payload, (query as any).payload.planFormat, (query as any).payload.format, 'default').pipe(
+ map(sqlMetadataQueryResult => new sqlQueryActions.ExecuteMetadataQuerySuccess(sqlMetadataQueryResult)),
+ catchError(sqlMetadataQueryError => of(new sqlQueryActions.ExecuteMetadataQueryFail(sqlMetadataQueryError)))
+ )
+ })
+ );
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/cancel.reducer.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/cancel.reducer.ts
new file mode 100644
index 0000000..2738139
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/cancel.reducer.ts
@@ -0,0 +1,56 @@
+/*
+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.
+*/
+import * as sqlCancelActions from '../actions/cancel.actions';
+
+export type Action_type = sqlCancelActions.All;
+
+export interface State {
+ currentRequestId: string,
+ success: boolean
+}
+
+const initialState: State = {
+ currentRequestId: "",
+ success: false
+}
+
+/*
+** Reducer function for sql++ queries in store/state
+*/
+export function cancelReducer(state = initialState, action: Action_type) {
+ switch (action.type) {
+ case sqlCancelActions.CANCEL_QUERY: {
+ return Object.assign({}, state, {
+ currentRequsetId: action.payload.requestId,
+ success: false
+ })
+ }
+
+ case sqlCancelActions.CANCEL_QUERY_SUCCESS: {
+ return Object.assign({}, state, {
+ success: true
+ })
+ }
+
+ case sqlCancelActions.CANCEL_QUERY_FAIL: {
+ return Object.assign({}, state, {
+ success: false
+ })
+ }
+
+ default: {
+ return state;
+ }
+ }
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/dataset.reducer.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/dataset.reducer.ts
index deaceb3..6edc206 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/dataset.reducer.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/dataset.reducer.ts
@@ -30,7 +30,9 @@
dropDatasetError: any[],
dropDatasetSuccess: boolean,
dropDatasetFailed: boolean,
- guideSelectsDataset: string
+ guideSelectsDataset: string,
+ sample: {},
+ dataset: string
};
const initialState: State = {
@@ -45,7 +47,9 @@
dropDatasetError: [],
dropDatasetSuccess: false,
dropDatasetFailed: false,
- guideSelectsDataset: ""
+ guideSelectsDataset: "",
+ sample: {},
+ dataset: ""
};
/*
@@ -84,6 +88,26 @@
}
/*
+ * Change the load state to true to signal that a SELECT Query is ongoing
+ */
+ case DatasetAction.SAMPLE_DATASET: {
+ return Object.assign({}, state, { loading: true, dataset: action.payload.dataset });
+ }
+
+ /*
+ * Change the load state to false, and loaded to true to signal
+ * that a SELECT Query is successful and there is a sample available
+ * in the store.
+ */
+ case DatasetAction.SAMPLE_DATASET_SUCCESS: {
+ return Object.assign({}, state, {
+ loaded: true,
+ loading: false,
+ sample: {...state.sample, [state.dataset]: action.payload.results[0]}
+ })
+ }
+
+ /*
* Change the load state to true to signaling
* that a CREATE a Dataset Query is ongoing
*/
@@ -173,4 +197,4 @@
default:
return state;
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/function.reducer.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/function.reducer.ts
new file mode 100644
index 0000000..ea0db87
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/function.reducer.ts
@@ -0,0 +1,68 @@
+/*
+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.
+*/
+
+import * as FunctionAction from '../actions/function.actions';
+
+export type Action = FunctionAction.All;
+
+/*
+** Interfaces for functions in store/state
+*/
+export interface State {
+ loaded: boolean,
+ loading: boolean,
+ functions: any[]
+}
+
+const initialState: State = {
+ loaded: false,
+ loading: false,
+ functions: []
+}
+
+/*
+** Reducer function for functions in store/state
+*/
+export function functionReducer(state = initialState, action: Action) {
+ switch (action.type) {
+ /*
+ * Change the load state to true to signaling
+ * that a SELECT Query is ongoing
+ */
+ case FunctionAction.SELECT_FUNCTIONS: {
+ return Object.assign({}, state, {
+ loading: true
+ });
+ }
+
+ /*
+ * Change the load state to false, and loaded to true to signaling
+ * that a SELECT Query is success and there are functions available in the
+ * store
+ */
+ case FunctionAction.SELECT_FUNCTIONS_SUCCESS: {
+ return Object.assign({}, state, {
+ loaded: true,
+ loading: false,
+ functions: action.payload
+ });
+ }
+
+ /*
+ * Just returns the current store/state object
+ */
+ default:
+ return state;
+ }
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/index.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/index.ts
index 77b678c..1a7109c 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/index.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/index.ts
@@ -15,9 +15,11 @@
import * as fromDataset from './dataset.reducer';
import * as fromDatatype from './datatype.reducer';
import * as fromIndex from './index.reducer';
+import * as fromFunction from './function.reducer';
import * as fromQuery from './query.reducer';
import * as fromQueryMetadata from './query-metadata.reducer';
import * as fromAppState from './app.reducer';
+import * as fromCancel from './cancel.reducer';
/*
** Global Interfaces store/state
@@ -27,6 +29,7 @@
dataset: fromDataset.State,
datatype: fromDatatype.State,
index: fromIndex.State,
+ functions: fromFunction.State,
sqlQuery: fromQuery.State,
sqlMetadataQuery: fromQueryMetadata.State,
appState: fromAppState.State,
@@ -40,7 +43,9 @@
dataset: fromDataset.datasetReducer,
datatype: fromDatatype.datatypeReducer,
index: fromIndex.indexReducer,
+ functions: fromFunction.functionReducer,
sqlQuery: fromQuery.sqlReducer,
+ cancelQuery: fromCancel.cancelReducer,
sqlMetadataQuery: fromQueryMetadata.sqlMetadataReducer,
app: fromAppState.appReducer
-};
\ No newline at end of file
+};
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/query.reducer.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/query.reducer.ts
index 408172b..1d2226c 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/query.reducer.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/reducers/query.reducer.ts
@@ -26,12 +26,14 @@
errorHash: {},
sqlQueryString: string,
sqlQueryPlanFormat: string,
+ sqlQueryOutputFormat: string,
sqlQueryStringHash: {},
sqlQueryPlanFormatHash: {},
sqlQueryResultHash: {},
sqlQueryErrorHash: {},
sqlQueryPrepared: {},
- sqlQueryMetrics: {}
+ sqlQueryMetrics: {},
+ sqlQueryWarnings: {}
};
const initialState: State = {
@@ -42,12 +44,14 @@
errorHash: {},
sqlQueryString: "",
sqlQueryPlanFormat: "",
+ sqlQueryOutputFormat: "",
sqlQueryStringHash: {},
sqlQueryPlanFormatHash: {},
sqlQueryResultHash: {},
sqlQueryErrorHash: {},
sqlQueryPrepared: {},
- sqlQueryMetrics: {}
+ sqlQueryMetrics: {},
+ sqlQueryWarnings: {}
};
/*
@@ -66,6 +70,7 @@
sqlQueryStringHash: state.sqlQueryStringHash,
sqlQueryPlanFormatHash: { ...state.sqlQueryPlanFormatHash, [action.payload.editorId]: action.payload.planFormat },
sqlQueryMetrics: state.sqlQueryMetrics,
+ sqlQueryWarnings: state.sqlQueryWarnings,
currentRequestId: state.currentRequestId
});
}
@@ -81,6 +86,7 @@
sqlQueryResultHash: { ...state.sqlQueryResultHash, [action.payload.editorId]: {} },
sqlQueryErrorHash: { ...state.sqlQueryErrorHash, [action.payload.editorId]: [] },
sqlQueryMetrics: { ...state.sqlQueryMetrics, [action.payload.editorId]: {} },
+ sqlQueryWarnings: { ...state.sqlQueryWarnings, [action.payload.editorId]: []},
currentRequestId: action.payload.editorId
});
}
@@ -98,11 +104,13 @@
errorHash: { ...state.errorHash, [action.payload.requestId]: false },
sqlQueryString: action.payload.queryString,
sqlQueryPlanFormat: action.payload.planFormat,
+ sqlQueryOutputFormat: action.payload.format,
sqlQueryStringHash: { ...state.sqlQueryStringHash, [action.payload.requestId]: action.payload.queryString },
sqlQueryPlanFormatHash: { ...state.sqlQueryPlanFormatHash, [action.payload.requestId]: action.payload.planFormat },
sqlQueryResultHash: { ...state.sqlQueryResultHash, [action.payload.requestId]: [] },
sqlQueryErrorHash: { ...state.sqlQueryErrorHash, [action.payload.requestId]: [] },
sqlQueryMetrics: { ...state.sqlQueryMetrics, [action.payload.requestId]: [] },
+ sqlQueryWarnings: { ...state.sqlQueryWarnings, [action.payload.requestId]: []},
});
}
@@ -122,7 +130,8 @@
sqlQueryStringHash: { ...state.sqlQueryStringHash, [state.currentRequestId]: state.sqlQueryString },
sqlQueryResultHash: { ...state.sqlQueryResultHash, [state.currentRequestId]: action.payload },
sqlQueryErrorHash: { ...state.sqlQueryErrorHash, [state.currentRequestId]: [] },
- sqlQueryMetrics: { ...state.sqlQueryMetrics, [state.currentRequestId]: action.payload.metrics }
+ sqlQueryMetrics: { ...state.sqlQueryMetrics, [state.currentRequestId]: action.payload.metrics },
+ sqlQueryWarnings: { ...state.sqlQueryWarnings, [state.currentRequestId]: action.payload.warnings},
})
}
@@ -143,6 +152,7 @@
sqlQueryResultHash: { ...state.sqlQueryResultHash, [state.currentRequestId]: [] },
sqlQueryErrorHash: { ...state.sqlQueryErrorHash, [state.currentRequestId]: action.payload.errors },
sqlQueryMetrics: { ...state.sqlQueryMetrics, [state.currentRequestId]: [] },
+ sqlQueryWarnings: { ...state.sqlQueryWarnings, [action.payload.requestId]: []},
})
}
@@ -153,4 +163,4 @@
return state;
}
}
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/app/shared/services/async-query.service.ts b/asterixdb/asterix-dashboard/src/node/src/app/shared/services/async-query.service.ts
index 5366531..c7c815d 100755
--- a/asterixdb/asterix-dashboard/src/node/src/app/shared/services/async-query.service.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/app/shared/services/async-query.service.ts
@@ -14,8 +14,7 @@
import { Injectable } from '@angular/core';
import { HttpClient, HttpHeaders } from '@angular/common/http';
import { Observable } from 'rxjs';
-import 'rxjs/add/operator/map'
-import 'rxjs/add/operator/catch'
+import { map, catchError } from 'rxjs/operators';
import { environment } from '../../../environments/environment';
/*
@@ -25,9 +24,12 @@
/* Using local proxy if webpack and development */
var AsterixRestApiUrl = '/query-service';
+var AsterixDeleteApiUrl = '/admin/requests/running';
+
if (environment.production) {
var locationHost = self.location.host.split(':')
- var AsterixRestApiUrl = 'http://' + locationHost[0] + ':19002/query/service';
+ AsterixRestApiUrl = 'http://' + locationHost[0] + ':19002/query/service';
+ AsterixDeleteApiUrl = 'http://' + locationHost[0] + ':19002/admin/requests/running';
}
/*
@@ -36,6 +38,9 @@
@Injectable()
export class SQLService {
defaultPlanFormat='JSON';
+ defaultOutputFormat='JSON';
+ defaultClientContextID='default';
+
/*
* SQLQueryService constructor using
* HttpClient from Angular 5
@@ -48,7 +53,7 @@
*/
selectDataverses() : Observable<any> {
let query = "SELECT VALUE dv FROM Metadata.`Dataverse` dv"
- return this.executeSQLQuery(query, this.defaultPlanFormat);
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
}
/*
@@ -57,7 +62,15 @@
*/
selectDatasets() : Observable<any> {
let query = "SELECT VALUE ds FROM Metadata.`Dataset` ds"
- return this.executeSQLQuery(query, this.defaultPlanFormat);
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
+ }
+
+ /*
+ * sends a select sql++ query to sample the passed in dataset.
+ */
+ sampleDataset(dataset: string) : Observable<any> {
+ let query = "SELECT * FROM " + dataset + " LIMIT 1;"
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
}
/*
@@ -66,7 +79,7 @@
*/
selectDatatypes() : Observable<any> {
let query = "SELECT VALUE dt FROM Metadata.`Datatype` dt"
- return this.executeSQLQuery(query, this.defaultPlanFormat);
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
}
/*
@@ -75,7 +88,12 @@
*/
selectIndexes() : Observable<any> {
let query = "SELECT VALUE ix FROM Metadata.`Index` ix"
- return this.executeSQLQuery(query, this.defaultPlanFormat);
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
+ }
+
+ selectFunctions() : Observable<any> {
+ let query = "SELECT VALUE fn FROM Metadata.`Function` fn"
+ return this.executeSQLQuery(query, this.defaultPlanFormat, this.defaultOutputFormat, this.defaultClientContextID);
}
/*
@@ -175,19 +193,59 @@
status: string;
login(username: string, password: string): Observable<boolean>
*/
- executeSQLQuery(query: string, planFormat: string): Observable<any> {
+ executeSQLQuery(query: string, planFormat: string, outputFormat: string, clientContextID: string): Observable<any> {
const apiUrl = AsterixRestApiUrl;
- const headers = new HttpHeaders();
- headers.append('Content-Type', 'application/json');
+ let headers = new HttpHeaders();
+
+ if (outputFormat == 'CSV_header') {
+ headers = new HttpHeaders({"Accept": "text/csv; header=present", "Content-Type": "application/json"});
+ outputFormat = 'CSV';
+ }
+ else if (outputFormat == 'CSV') {
+ headers = new HttpHeaders({"Accept": "text/csv; header=absent", "Content-Type": "application/json"});
+ }
+ else
+ headers = headers.append('Content-Type', 'application/json');
+
let options = ({ headers: headers });
- return this.http.post(apiUrl, {statement: query,
- 'logical-plan': true,
- 'optimized-logical-plan': true,
- 'plan-format': planFormat }, options)
- .map((response: Response) => { return response; })
- .catch((error: any) => this.handleExecuteQueryError(error))
+ let body = {
+ statement: query,
+ 'logical-plan': true,
+ 'optimized-logical-plan': true,
+ 'plan-format': planFormat,
+ 'max-warnings': Number.MAX_SAFE_INTEGER
+ }
+
+ if (clientContextID != 'default') {
+ body['client_context_id'] = clientContextID;
+ }
+
+ return this.http.post(apiUrl, body, options)
+ .pipe(map((response: Response) => { return response; }),
+ catchError((error: any) => this.handleExecuteQueryError(error)))
+
+
}
+
+ /*
+ * Cancels a sql++ query
+ */
+ cancelSQLQuery(clientContextID: string): Observable<any> {
+ let url = AsterixDeleteApiUrl;
+ let headers = new HttpHeaders();
+
+ url += `?client_context_id=${clientContextID}`;
+ headers = headers.append('Content-Type', 'application/json');
+
+ let options = ({ headers: headers });
+
+ return this.http.delete(url, options)
+ .pipe(map((response: Response) => {return response; }),
+ catchError((error: any) => this.handleDeleteQueryError(error))
+ )
+ }
+
/*
* AsterixDB query-service API raises HTTP errors if the sql++ query has some
* syntax error, or some elements in the query are not found
@@ -204,4 +262,10 @@
console.log(error);
return Promise.reject(error.error || error);
}
-}
\ No newline at end of file
+
+ private handleDeleteQueryError(error: any): Promise<any> {
+ console.log('deleteQueryError:')
+ console.log(error);
+ return Promise.reject(error.error || error);
+ }
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/index.html b/asterixdb/asterix-dashboard/src/node/src/index.html
index 0d264bc..e790e8c 100755
--- a/asterixdb/asterix-dashboard/src/node/src/index.html
+++ b/asterixdb/asterix-dashboard/src/node/src/index.html
@@ -15,7 +15,7 @@
<html>
<head>
<meta charset="utf-8">
- <title>AsterixDb Administration Console</title>
+ <title>AsterixDB Admin Console</title>
<base href="/">
<link rel="icon" type="image/x-icon" href="favicon.ico">
</head>
diff --git a/asterixdb/asterix-dashboard/src/node/src/main.scss b/asterixdb/asterix-dashboard/src/node/src/main.scss
index 0a67dd2..c18bdb4 100755
--- a/asterixdb/asterix-dashboard/src/node/src/main.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/main.scss
@@ -15,6 +15,8 @@
// Include material core styles.
@import '~@angular/material/theming';
@include mat-core();
+// Include codemirror styles
+@import '~codemirror/addon/hint/show-hint';
// Define the light theme.
$primary: mat-palette($mat-grey);
$accent : mat-palette($mat-orange, A200, A100, A400);
diff --git a/asterixdb/asterix-dashboard/src/node/src/main.ts b/asterixdb/asterix-dashboard/src/node/src/main.ts
index ac7552b..848666d 100755
--- a/asterixdb/asterix-dashboard/src/node/src/main.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/main.ts
@@ -23,4 +23,4 @@
}
platformBrowserDynamic().bootstrapModule(AppModule)
- .catch(err => console.log(err));
\ No newline at end of file
+ .catch(err => console.log(err));
diff --git a/asterixdb/asterix-dashboard/src/node/src/polyfills.ts b/asterixdb/asterix-dashboard/src/node/src/polyfills.ts
index b7ac983..b8fb265 100755
--- a/asterixdb/asterix-dashboard/src/node/src/polyfills.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/polyfills.ts
@@ -43,7 +43,7 @@
/** Evergreen browsers require these. **/
// Used for reflect-metadata in JIT. If you use AOT (and only Angular decorators), you can remove.
-import 'core-js/es7/reflect';
+//import 'core-js/es7/reflect';
/**
@@ -73,4 +73,4 @@
/**
* Need to import at least one locale-data with intl.
*/
-// import 'intl/locale-data/jsonp/en';
\ No newline at end of file
+// import 'intl/locale-data/jsonp/en';
diff --git a/asterixdb/asterix-dashboard/src/node/src/styles/_general.scss b/asterixdb/asterix-dashboard/src/node/src/styles/_general.scss
index ef2d7a2..fa4c40c 100755
--- a/asterixdb/asterix-dashboard/src/node/src/styles/_general.scss
+++ b/asterixdb/asterix-dashboard/src/node/src/styles/_general.scss
@@ -81,9 +81,9 @@
top: 0 !important;
}
-.CodeMirror {
- font-family: Arial, monospace;
- font-size: 14px !important;
+.CodeMirror * {
+ font-family: "Courier New", monospace !important;
+ font-size: 16px !important;
color: black;
}
@@ -93,4 +93,4 @@
.mat-expansion-panel-header-description {
margin-right: 0px !important;
-}
\ No newline at end of file
+}
diff --git a/asterixdb/asterix-dashboard/src/node/src/test.ts b/asterixdb/asterix-dashboard/src/node/src/test.ts
index 06c7693..4d9f2f5 100755
--- a/asterixdb/asterix-dashboard/src/node/src/test.ts
+++ b/asterixdb/asterix-dashboard/src/node/src/test.ts
@@ -13,34 +13,26 @@
*/
// This file is required by karma.conf.js and loads recursively all the .spec and framework files
-
-import 'zone.js/dist/long-stack-trace-zone';
-import 'zone.js/dist/proxy.js';
-import 'zone.js/dist/sync-test';
-import 'zone.js/dist/jasmine-patch';
-import 'zone.js/dist/async-test';
-import 'zone.js/dist/fake-async-test';
+import 'zone.js/dist/zone-testing';
import { getTestBed } from '@angular/core/testing';
import {
BrowserDynamicTestingModule,
platformBrowserDynamicTesting
} from '@angular/platform-browser-dynamic/testing';
-// Unfortunately there's no typing for the `__karma__` variable. Just declare it as any.
-declare const __karma__: any;
-declare const require: any;
-
-// Prevent Karma from running prematurely.
-__karma__.loaded = function () {};
+declare const require: {
+ context(path: string, deep?: boolean, filter?: RegExp): {
+ keys(): string[];
+ <T>(id: string): T;
+ };
+};
// First, initialize the Angular testing environment.
getTestBed().initTestEnvironment(
- BrowserDynamicTestingModule,
- platformBrowserDynamicTesting()
+ BrowserDynamicTestingModule,
+ platformBrowserDynamicTesting()
);
// Then we find all the tests.
const context = require.context('./', true, /\.spec\.ts$/);
// And load the modules.
context.keys().map(context);
-// Finally, start Karma to run the tests.
-__karma__.start();
\ No newline at end of file
diff --git a/asterixdb/asterix-dashboard/src/node/src/tests/appbar.component.spec.ts b/asterixdb/asterix-dashboard/src/node/src/tests/appbar.component.spec.ts
new file mode 100644
index 0000000..a62aa4e
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/tests/appbar.component.spec.ts
@@ -0,0 +1,68 @@
+/*
+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.
+*/
+
+import { AppBarComponent } from "../app/dashboard/appbar.component";
+import {TestBed, waitForAsync} from "@angular/core/testing";
+import {StoreModule} from "@ngrx/store";
+import {CUSTOM_ELEMENTS_SCHEMA} from "@angular/core";
+
+describe('AppBarComponent', () => {
+ beforeEach(waitForAsync(() => {
+ TestBed.configureTestingModule({
+ declarations: [
+ AppBarComponent
+ ],
+ imports: [
+ StoreModule.forRoot({})
+ ],
+ schemas: [
+ CUSTOM_ELEMENTS_SCHEMA
+ ]
+ }).compileComponents();
+ }));
+
+ it('should render link titled "WEBSITE"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(1)').textContent).toContain('WEBSITE');
+ }));
+
+ it('should render link titled "FILE ISSUES"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(2)').textContent).toContain('FILE ISSUES');
+ }));
+
+ it('should render link titled "DOCUMENTATION"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(3)').textContent).toContain('DOCUMENTATION');
+ }));
+
+ it('should render link titled "CONTACT"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(4)').textContent).toContain('CONTACT');
+ }));
+
+ it('should render link titled "GITHUB"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(5)').textContent).toContain('GITHUB');
+ }));
+
+ it('should render link titled "METADATA"', waitForAsync(() => {
+ const fixture = TestBed.createComponent(AppBarComponent);
+ const compiled = fixture.debugElement.nativeElement;
+ expect(compiled.querySelector('.menu:nth-child(6)').textContent).toContain('METADATA');
+ }));
+});
diff --git a/asterixdb/asterix-dashboard/src/node/src/tests/input.component.spec.ts b/asterixdb/asterix-dashboard/src/node/src/tests/input.component.spec.ts
new file mode 100644
index 0000000..d006b44
--- /dev/null
+++ b/asterixdb/asterix-dashboard/src/node/src/tests/input.component.spec.ts
@@ -0,0 +1,213 @@
+/*
+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.
+*/
+import {InputQueryComponent} from "../app/dashboard/query/input.component";
+import {ComponentFixture, TestBed, waitForAsync, async} from "@angular/core/testing";
+import {provideMockStore, MockStore} from "@ngrx/store/testing";
+import {CUSTOM_ELEMENTS_SCHEMA, DebugElement} from "@angular/core";
+import {By} from "@angular/platform-browser";
+import {HarnessLoader} from "@angular/cdk/testing";
+import {TestbedHarnessEnvironment} from "@angular/cdk/testing/testbed";
+
+describe('InputQueryComponent: unit test', () => {
+ let component: InputQueryComponent;
+ let fixture: ComponentFixture<InputQueryComponent>;
+ let de: DebugElement;
+ let el: HTMLElement;
+ let loader: HarnessLoader;
+
+ let store: MockStore;
+ const initialState = {
+ sqlQuery: {
+ currentRequestId: '',
+ loadingHash: {},
+ loadedHash: {},
+ successHash:{},
+ errorHash: {},
+ sqlQueryString: "",
+ sqlQueryPlanFormat: "",
+ sqlQueryOutputFormat: "",
+ sqlQueryStringHash: {},
+ sqlQueryPlanFormatHash: {},
+ sqlQueryResultHash: {},
+ sqlQueryErrorHash: {},
+ sqlQueryPrepared: {},
+ sqlQueryMetrics: {},
+ sqlQueryWarnings: {}
+ },
+ cancelQuery: {
+ currentRequestId: "",
+ success: false
+ },
+ dataverse: {
+ loaded: false,
+ loading: false,
+ dataverses: [],
+ createDataverse: [],
+ createDataverseName: "",
+ createDataverseError: [],
+ createDataverseSuccess: false,
+ createDataverseFailed: false,
+ dropDataverse: [],
+ dropDataverseName: "",
+ dropDataverseError: [],
+ dropDataverseSuccess: false,
+ dropDataverseFailed: false,
+ defaultDataverseName: 'Default'
+ }
+ }
+
+ beforeEach(async() => {
+ await TestBed.configureTestingModule({
+ declarations: [
+ InputQueryComponent
+ ],
+ providers: [
+ provideMockStore({initialState})
+ ],
+ schemas: [CUSTOM_ELEMENTS_SCHEMA]
+ }).compileComponents();
+
+ store = TestBed.inject(MockStore);
+ spyOn(store, 'dispatch');
+ jasmine.DEFAULT_TIMEOUT_INTERVAL = 30000;
+ });
+
+ beforeEach(() => {
+ fixture = TestBed.createComponent(InputQueryComponent);
+ component = fixture.componentInstance;
+ de = fixture.debugElement;
+ fixture.detectChanges();
+ loader = TestbedHarnessEnvironment.loader(fixture);
+ });
+
+ it('should create', () => {
+ expect(component).toBeTruthy();
+ })
+
+ //query navigation
+ describe('query navigation tests', () => {
+ it('running query should call store.dispatch 3 times', () => {
+ expect(store.dispatch).toHaveBeenCalledTimes(3);
+
+ component.queryString = "select 1";
+ component.onClickRun();
+ fixture.detectChanges();
+ expect(store.dispatch).toHaveBeenCalledTimes(6);
+
+ component.queryString = "select 2";
+ component.onClickRun();
+ fixture.detectChanges();
+ expect(store.dispatch).toHaveBeenCalledTimes(9);
+
+ component.queryString = "select 3";
+ component.onClickRun();
+ fixture.detectChanges();
+ expect(store.dispatch).toHaveBeenCalledTimes(12);
+ });
+ });
+
+ //outputs
+ describe('test outputs', () => {
+ it('emits isExplain is false on click run', () => {
+ let actualInputToOutput = {};
+
+ component.inputToOutputEmitter.subscribe((inputToOutput) => {
+ actualInputToOutput = inputToOutput;
+ })
+
+ let button = de.query(By.css('.run'));
+ button.nativeElement.click();
+
+ expect(actualInputToOutput['isExplain']).toBe(false);
+ });
+
+ it('emits isExplain is true on click explain', () => {
+ let actualInputToOutput: boolean = undefined;
+
+ component.inputToOutputEmitter.subscribe((inputToOutput) => {
+ actualInputToOutput = inputToOutput['isExplain'];
+ })
+
+ let button = de.query(By.css('.explain'));
+ button.nativeElement.click();
+
+ expect(actualInputToOutput).toBe(true);
+ });
+
+ it('emits JSON output format when selected from dropdown', async()=> {
+ let outputFormat: string = undefined;
+
+ component.inputToOutputEmitter.subscribe((inputToOutput) => {
+ outputFormat = inputToOutput['outputFormat'];
+ })
+
+ component.outputOptions = 'JSON';
+
+ let button = de.query(By.css('.run'));
+ button.nativeElement.click();
+
+ expect(outputFormat).toEqual('JSON');
+ });
+
+ it('emits CSV (no header) output format when selected from dropdown', async() => {
+ let outputFormat: string = undefined;
+
+ component.inputToOutputEmitter.subscribe((inputToOutput) => {
+ outputFormat = inputToOutput['outputFormat'];
+ })
+
+ component.outputOptions = 'CSV';
+
+ let button = de.query(By.css('.run'));
+ button.nativeElement.click();
+
+ expect(outputFormat).toEqual('CSV');
+ });
+
+ it('emits CSV (header) output format when selected from dropdown', () => {
+ let outputFormat: string = undefined;
+
+
+ component.inputToOutputEmitter.subscribe((inputToOutput) => {
+ outputFormat = inputToOutput['outputFormat'];
+ })
+
+ component.outputOptions = 'CSV_header'
+
+ let button = de.query(By.css('.run'));
+ button.nativeElement.click();
+
+ expect(outputFormat).toEqual('CSV_header');
+ });
+ });
+
+ //clear tests
+ describe('clear() tests', () => {
+ it('clicking #clear() should clear query string', () => {
+ component.queryString = "select 1";
+ expect(component.queryString).toBe("select 1");
+ component.onClickClear();
+ expect(component.queryString).toBe("");
+ });
+
+ it('clicking clear should clear the editor on screen', async() => {
+ component.editor.getDoc().setValue("this is a test text");
+
+ let button = de.query(By.css('.clear'));
+ button.nativeElement.click();
+
+ expect(component.editor.getDoc().getValue()).toEqual("");
+ expect(component.queryString).toEqual("");
+ });
+
+ });
+});
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index c58d983..a5c8aed 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<url>http://asterixdb.apache.org/</url>
@@ -52,7 +52,7 @@
<configuration>
<target>
<concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/manual.md">
- <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_query.md,4_windowfunctions_title.md,4_windowfunctions.md,5_error_title.md,5_error.md,6_sql_diff_title.md,6_sql_diff.md,7_ddl_head.md,7_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_2_hints.md,appendix_3_title.md,appendix_3_resolution.md,appendix_4_title.md,appendix_4_manual_data.md" />
+ <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_query.md,4_windowfunctions_title.md,4_windowfunctions.md,5_error_title.md,5_error.md,6_sql_diff_title.md,6_sql_diff.md,7_ddl_head.md,7_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_2_arrayindex.md,appendix_2_hints.md,appendix_3_title.md,appendix_3_resolution.md,appendix_4_title.md,appendix_4_manual_data.md" />
</concat>
<concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/builtins.md">
<filelist dir="${project.basedir}/src/main/markdown/builtins" files="0_toc.md,0_toc_sqlpp.md,0_toc_common.md,1_numeric_common.md,1_numeric_delta.md,2_string_common.md,2_string_delta.md,3_binary.md,4_spatial.md,5_similarity.md,6_tokenizing.md,7_temporal.md,7_allens.md,8_record.md,9_aggregate_sql.md,10_comparison.md,11_type_common.md,11_type_delta.md,11_type_conversion.md,13_conditional.md,12_misc.md,15_bitwise.md,14_window.md" />
diff --git a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
index cf7d4a5..db89fe0 100644
--- a/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
+++ b/asterixdb/asterix-doc/src/main/grammar/sqlpp.ebnf
@@ -210,10 +210,17 @@
CreateIndex ::= CreateSecondaryIndex | CreatePrimaryKeyIndex
CreateSecondaryIndex ::= "CREATE" "INDEX" Identifier ("IF" "NOT" "EXISTS")? "ON" QualifiedName
- "(" IndexField ( "," IndexField )* ")" ("TYPE" IndexType)? ("ENFORCED")?
+ "(" IndexedElement ( "," IndexedElement )* ")" ("TYPE" IndexType)? ("ENFORCED")?
CreatePrimaryKeyIndex ::= "CREATE" "PRIMARY" "INDEX" Identifier? ("IF" "NOT" "EXISTS")? "ON" QualifiedName ("TYPE" "BTREE")?
+IndexedElement ::= ArrayIndexElement
+ | IndexField
+ | "(" ( ArrayIndexElement | IndexField ) ")"
+
+ArrayIndexElement ::= "UNNEST" NestedField ( "UNNEST" NestedField )*
+ ( ( ":" TypeReference ) | ( "SELECT" NestedField ( ":" TypeReference )? ( "," NestedField ( ":" TypeReference )? )* ) )?
+
IndexField ::= NestedField ( ":" TypeReference "?"? )?
IndexType ::= "BTREE"
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
index f32d603..1569d69 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/7_ddl_dml.md
@@ -304,8 +304,14 @@
##### CreatePrimaryKeyIndex
![](../images/diagrams/CreatePrimaryKeyIndex.png)
+##### IndexedElement
+**![](../images/diagrams/IndexedElement.png)**
+
+##### ArrayIndexElement
+**![](../images/diagrams/ArrayIndexElement.png)**
+
##### IndexField
-![](../images/diagrams/IndexField.png)
+**![](../images/diagrams/IndexField.png)**
##### NestedField
![](../images/diagrams/NestedField.png)
@@ -317,6 +323,8 @@
Supported index types include `BTREE` for totally ordered datatypes, `RTREE` for spatial data,
and `KEYWORD` and `NGRAM` for textual (string) data.
An index can be created on a nested field (or fields) by providing a valid path expression as an index field identifier.
+An array index can be created on an array or multiset datatype by providing a sequence of `UNNEST` and `SELECT`s to
+identify the field(s) to be indexed.
An indexed field is not required to be part of the datatype associated with a dataset if the dataset's datatype
is declared as open **and** if the field's type is provided along with its name and if the `ENFORCED` keyword is
@@ -355,12 +363,19 @@
a nested field residing within a object-valued user field in the `orders` dataset.
This index can be useful for accelerating exact-match queries, range search queries,
and joins involving the nested `orderUserName` field.
-Such nested fields must be singular, i.e., one cannot index through (or on) an array-valued field.
##### Example
CREATE INDEX oOrderUserNameIdx ON orders(order.orderUserName) TYPE BTREE;
+The following example creates an array index called `oItemsPriceIdx` on the `price` field inside the `items` array of the `orders` dataset.
+This index can be useful for accelerating membership queries, existential or universal quantification queries, or joins involving the `price` field inside this array.
+(To enable array index query optimization, be sure to set the [`arrayindex` compiler option](manual.html#ArrayIndexFlag).)
+
+#### Example
+
+ CREATE INDEX oItemsPriceIdx ON orders(UNNEST items SELECT price);
+
The following example creates an open rtree index called `oOrderLocIdx` on the order-location field of the `orders` dataset. This index can be useful for accelerating queries that use the [`spatial-intersect` function](builtins.html#spatial_intersect) in a predicate involving the sender-location field.
##### Example
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
new file mode 100644
index 0000000..ee90efb
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_arrayindex.md
@@ -0,0 +1,34 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you 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.
+ !-->
+
+## <a id="ArrayIndexFlag">Controlling Array-Index Access Method Plan Parameter</a>
+By default, the system does not attempt to utilize array indexes as an access method (even if an array index is present and is applicable).
+If you believe that your query will benefit from an array index, toggle the parameter below.
+
+* **compiler.arrayindex**: if this is set to true, array indexes will be considered as an access method for applicable queries; the default value is false.
+
+
+#### Example
+
+ set `compiler.arrayindex` "true";
+
+ SELECT o.orderno
+ FROM orders o
+ WHERE SOME i IN o.items
+ SATISFIES i.price = 19.91;
diff --git a/asterixdb/asterix-doc/src/main/user-defined_function/udf.md b/asterixdb/asterix-doc/src/main/user-defined_function/udf.md
index 655113b..71e0fa2 100644
--- a/asterixdb/asterix-doc/src/main/user-defined_function/udf.md
+++ b/asterixdb/asterix-doc/src/main/user-defined_function/udf.md
@@ -60,7 +60,7 @@
* `udfs` dataverse name
* with a new Library name of `testlib`
* from `lib.zip` in the present working directory
-* to the cluster at `localhost` with API port `19002`
+* to the cluster at `localhost` with API port `19004` of the Asterix CC
* with credentials being a username and password of `admin:admin`
we would execute
@@ -71,7 +71,13 @@
In the AsterixDB source release, we provide several sample UDFs that you can try out.
You need to build the AsterixDB source to get the compiled UDF package. It can be found under
-the `asterixdb-external` sub-project. Assuming that these UDFs have been installed into the `testlib` library in`udfs` dataverse,
+the `asterix-external-data` sub-project under the path
+`asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library`.
+After compilation, the UDFs will be packed in a zip file at
+`asterixdb/asterix-external-data/target/asterix-external-data-$VERSION-testlib.zip`
+which you can use to upload to the AsterixDB cluster.
+
+Assuming that these UDFs have been installed into the `testlib` library in`udfs` dataverse,
here is an example that uses the sample UDF `mysum` to compute the sum of two input integers.
USE udfs;
diff --git a/asterixdb/asterix-doc/src/site/markdown/dashboard.md b/asterixdb/asterix-doc/src/site/markdown/dashboard.md
new file mode 100644
index 0000000..2d6a519
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/markdown/dashboard.md
@@ -0,0 +1,236 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you 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.
+ !-->
+
+ # AsterixDB Administration Console #
+
+ ## <a id="toc">Table of Contents</a>
+
+ * [Basics](#basics)
+ * [Query Navigation](#qnav)
+ * [Metadata Inspector](#metadatainspector)
+ * [Interactive Plan Viewer](#planviewer)
+ * [Exporting Data](#exporting)
+ * [Development](#development)
+
+## <a id="basics">Basic Usage</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+Executing a query on this console is easy. First, select from the input options and then select your execution mode.
+
+__Input Options__
+
+* `Dataverse` - the dataverse that the query will use. The default is the `Default` dataverse. This is not required to
+run a query and the console will try and autodetect the dataverse used in the query.
+* `Plan Format` - specifies what format of the resulting query plan.
+ * `JSON` - results in the showing the interactive query plan viewer.
+ * `STRING` - results in the text/string format of the query plan. Equivalent to the text format from the old 19001
+ console.
+* `Output Format` - the format of the result of the query.
+ * `JSON` - the default and will return the results in JSON. Can also view in `Tree` and `Table` views in the output
+ section.
+ * `CSV (no header)` - will return CSV format but without the header. Can only view this in `Table` view in the output
+ section.
+ * `CSV (header)` - will return CSV format with the header. Can only view this in `Table` view in the output
+ section. See the [Exporting Data](#exporting) section for more information and examples.
+
+To execute the query click the green triangle in the bottom right. Users may also choose to click the `Explain` button.
+This will not actually run the query (no results returned) and will only return the query plan. The console will default
+the view in the output section to `Plan`.
+
+To cancel the query click the red stop button in the bottom right. This will send a `DELETE` request to the server and cancel the previous
+request.
+
+The dashboard now supports autocomplete for SQL++ keywords. Use `CTRL+Space` to activate the autocomplete.
+
+## <a id="qnav">Query Navigation</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+This console supports query history and has two different ways of navigating the query history. On the input bar there is a section
+for `QUERY HISTORY` and there are also two arrows `<` and `>`.
+
+Utilizing the arrows will let you traverse the queries one by one. However, if the console is already at the most recent query
+in the history and the user clicks the `>` or forward arrow, it will create a new empty query.
+
+The `QUERY HISTORY` dropdown allows users to jump through the history without having to step through it with the arrows.
+
+When executing a query, this query will be counted as a new query if it is different (purely the text of the query, not
+the results) from the most recent query. It will subsequently be added to the query history.
+
+## <a id="metadatainspector">Metadata Inspector</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+The metadata inspector is the column on the rightside of the console. The `Refresh` button is used to update the current metadata.
+When a user creates or drops a Dataverse, Dataset, Datatype, or Index the changes will not be automatically reflected. User must
+click the `Refresh` button to get the most up to date data.
+
+The console supports multiple dialogs/windows open at once. All of these are resizable and draggable as well.
+
+Users can also click the `JSON` / `SUMMARY` button to toggle from the raw and parsed views. `SUMMARY` is the default.
+
+#### Dataverse
+
+Clicking a dataverse will add it to the shown metadata in this inspector. Users can select as many dataverses as desired.
+The corresponding datasets, datatypes, and indices will appear.
+
+#### Datasets
+
+Clicking on a dataset will open a draggable and expandable window that contains information about the dataset.
+
+* `Dataverse` - which dataverse the dataset belongs to.
+* `Dataset` - the name of the dataset.
+* `Datatype Name` - the name of the datatype of the dataset.
+* `Primary Keys` - the primary keys of the dataset.
+* `Sample` - if there is data inserted into the dataset, this is a section where viewers can see a sample of the dataset.
+It is a `SELECT * FROM {dataset} LIMIT 1` query.
+
+#### Datatypes
+
+Clicking on a datatypes will open a draggable and expandable window that contains information about the datatype. This console
+does support nested datatypes.
+
+* `Dataverse` - which dataverse the datatype belongs to.
+* `Datatype Name` - the name of the datatype.
+* `Fields` - a list of the fields in the dataset. Each field has information on whether it is nullable or required. If the
+field is nested / not a primitive type, click on it to see the information of that type. If the field is wrapped in `[ ]` or `{{ }}`,
+then it is an ordered list or unordered of that type respectively. If a field is italicized, it is an anonymous type.
+
+NOTE: the `JSON` view does not support nested like `SUMMARY` does.
+
+#### Index
+
+Clicking on a dataset will open a draggable and expandable window that contains information about the index.
+
+* `Dataverse` - which dataverse the index belongs to.
+* `Index Name` - the name of the index.
+* `Index Type` - the type of the index.
+* `Search Key(s)` - the key(s) of the index.
+
+
+
+## <a id="planviewer">Interactive Plan Viewer</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+To view the interactive plan viewer, execute a query and switch to the `PLAN` tab in the output section. Alternatively,
+users can click explain the query by clicking `EXPLAIN` instead of execute and the output section will default to the
+`PLAN` tab.
+
+To interact with the query plan, drag to move the view of the graph. Use the scroll wheel or scroll movement to zoom in and out
+of the plan.
+
+The default plan orientation is `Bottom to Top` and can be swapped for `Top to Bottom` if so desired.
+
+The default view of the plan is not detailed (just operator IDs and operator names). To look at a more detailed plan, check
+the `Detailed` checkbox and the plan will reload with more detail per node.
+
+#### Traversing
+
+There are multiple ways to traverse the query plan. the `Go to Node` dropdown will keep track of the currently selected
+node. Using the arrows next to the `Go to Node` dropdown will traverse the plan node by node in a Depth First Search (DFS)
+fashion. Selecting nodes on the `Go to Node` dropdown will jump the plan to the selected node.
+
+Utilizing both the arrows and the `Go to Node` dropdown, it is easy to trace through a plan.
+
+#### Search (Detailed mode only)
+
+The `Search` function appears when the plan is in `Detailed` mode. Search for specific string occurrences in the plan. When
+the search icon is clicked, the first mathc will be selected (if there is a match). Use the arrows that appear next to it
+to iterate through every match.
+
+Must click `Clear Selections` after done with the search.
+
+Unfortunately, at this time regular expression search is not supported.
+
+#### Variables (Detailed mode only)
+
+The `See Variable Occurences` dropdown will appear when the plan is in `Detailed` mode. Users can select from any variable
+that appears in the plan. Selecting a variable will jump to the node of last occurrence. The user can see how many occurence there
+are by the `See Variable Occurences` dropdown title (it will now include a fraction).
+
+The arrows that appear can iterate through the occurences.
+
+Often, it is useful to be able to skip right to the declaration of a variable. By clicking on the skip button, the plan
+will select the node where that variable was declared. To jump back to whatever node before, click the undo button.
+
+#### Clear Selections
+
+Clicking `Clear Selections` will reset the graph and focus on the first node in the plan.
+
+## <a id="exporting">Exporting Data</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+### JSON/JSONL:
+
+1. Select `JSON` in the input `Output Format` option and run the query that you want to export.
+2. Click `Export` in the output section.
+3. Select between `JSON` and `JSONL` (JSON Lines). Adjust the filename to the desired name.
+4. Click `Export` to start the download.
+
+### CSV (no header):
+
+1. Select `CSV (no header)` in the input `Output Format` option and run the query that you want to export.
+2. Click `Export` in the output section.
+3. Adjust the filename to the desired name.
+4. Click `Export` to start the download.
+
+### CSV (header):
+
+1. Create a type that supports the query you want to run.
+2. Set the `output-record-type` before your query
+3. Select `CSV (no header)` in the input `Output Format` option and run the query that you want to export.
+4. Click `Export` in the output section.
+5. Adjust the filename to the desired name.
+6. Click `Export` to start the download.
+
+This one is trickier. In order to get the header in the CSV format, we need to set the `output-record-type` in the query
+in order to get the headers. To explain further, here is an example using the TinySocial dataset from the Using SQL++ Primer.
+
+ CREATE TYPE GleambookMessageType AS {
+ messageId: int,
+ authorId: int,
+ inResponseTo: int?,
+ senderLocation: point?,
+ message: string
+ };
+
+ CREATE DATASET GleambookMessages(GleambookMessageType)
+ PRIMARY KEY messageId;
+
+If we wanted to export `messageId`, `authorId`, and `senderLocation` in CSV format with headers, we would have to create
+an additional type to support this export.
+
+ CREATE TYPE GleambookMessages_exportCSV AS {
+ messageId: int,
+ authorId: int,
+ senderLocation: point
+ };
+
+The query would then look something like this:
+
+ USE TinySocial;
+
+ SET `output-record-type` "GleambookMessages_exportCSV";
+
+ SELECT messageId, authorId, senderLocation
+ FROM GleambookMessages;
+
+Now run the query with the `CSV (header)` input option and the result will contain the hedaer `messageId`, `authorId`,
+and `senderLocation`.
+
+## <a id="development">Development</a><font size="4"> <a href="#toc">[Back to TOC]</a></font>
+
+To start the development server, run `ng serve` or `npm start`. Navigate to `http://localhost:4200/`.
+The app will automatically reload if you change any of the source files.
+
+To add a debugger, add a new `Javascrip Debug` configuration in the IntelliJ `Run Configurations` and set the URL to
+`http://localhost:4200/`. Additionally, you can set the file directory to asterix-dashboard.
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
new file mode 100644
index 0000000..559e2d2
--- /dev/null
+++ b/asterixdb/asterix-doc/src/site/markdown/sqlpp/arrayindex.md
@@ -0,0 +1,188 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you 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.
+ !-->
+
+# AsterixDB Support of Array Indexes #
+
+## <a id="toc">Table of Contents</a> ##
+
+* [Overview](#Overview)
+* [Quantification Queries](#QuantificationQueries)
+* [Explicit Unnesting Queries](#ExplicitUnnestQueries)
+* [Join Queries](#JoinQueries)
+* [Complex Indexing Examples](#ComplexIndexingExamples)
+
+
+## <a id="Overview">Overview</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Array indexes are used in applications where users want to accelerate a query that involves some array-valued or multiset-valued field.
+This enables fast evaluation of predicates in queries involving arrays or multisets in datasets.
+For brevity, all further mentions of array-valued fields are also applicable to multiset-valued fields.
+
+Array-valued fields are a natural data modeling concept for documents.
+In the traditional inventory management example, it is natural for the line items of an order to exist as a part of the order itself.
+Previously if an AsterixDB user wanted to optimize a query involving a predicate on the line items of an order, they would a) have to undertake some form of schema migration to separate the line items from the orders into different datasets, b) create an index on the new dataset for line items, and finally c) modify their query to join orders and line items.
+With the introduction of array indexes in AsterixDB, users can keep their arrays intact and still reap the performance benefits of an index.
+
+It should be noted that in AsterixDB, array indexes are *not* meant to serve as covering indexes.
+In fact due to AsterixDB's record-level locking, index-only plans involving multi-valued fields (i.e. array indexes and inverted indexes) are not currently possible.
+Instead, array indexes are simply meant to accelerate queries involving multi-valued fields.
+
+
+## <a id="QuantificationQueries">Quantification Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+A common use-case for array indexes involves quantifying some or all elements within an array.
+Quantification queries have two variants: existential and universal.
+Existential queries ask if *any* element in some array satisfies a given predicate.
+Membership queries are a specific type of existential query, asking if any element in some array is equal to a particular value.
+Universal queries ask if *all* elements in some array satisfy a particular predicate.
+Empty arrays are not stored in an array index, meaning that a user must additionally specify that the array is non-empty to tell AsterixDB that it is possible to use an array index as an access method for the given query.
+
+All query examples here will use the orders and products datasets below.
+
+ CREATE TYPE ordersType AS {
+ orderno: int,
+ custid: string,
+ items: [{ itemno: int, productno: int, qty: int, price: float }]
+ };
+ CREATE DATASET orders (ordersType) PRIMARY KEY orderno;
+
+ CREATE TYPE productsType AS {
+ productno: int,
+ categories: {{ string }}
+ };
+ CREATE DATASET products (productsType) PRIMARY KEY productno;
+
+Let us now create an index on the `categories` multiset of the `products` dataset.
+
+ CREATE INDEX pCategoriesIdx ON products (UNNEST categories);
+
+Suppose we now want to find all products that have the category "Food".
+The following membership query will utilize the index we just created.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT p
+ FROM products p
+ WHERE "Food" IN p.categories;
+
+We can also rewrite the query above as an explicit existential quantification query with an equality predicate and the index will be utilized.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT p
+ FROM products p
+ WHERE SOME c IN p.categories SATISFIES c = "Food";
+
+Let us now create an index on the `qty` and `price` fields in the `items` array of the `orders` dataset.
+
+ CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty, price);
+
+Now suppose we want to find all orders that only have items with large quantities and low prices, not counting orders without any items.
+The following universal quantification query will utilize the index we just created.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT o
+ FROM orders o
+ WHERE LEN(o.items) > 0 AND
+ (EVERY i IN o.items SATISFIES i.qty > 100 AND i.price < 5.00);
+
+Take note of the `LEN(o.items) > 0` conjunct.
+Array indexes cannot be used for queries with potentially empty arrays.
+
+
+## <a id="ExplicitUnnestQueries">Explicit Unnesting Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Array indexes can also be used to accelerate queries that involve the explicit unnesting of array fields.
+We can express the same membership / existential example above using an explicit `UNNEST` query.
+(To keep the same cardinality as the query above (i.e. to undo the `UNNEST`), we add a `DISTINCT` clause, though the index would be utilized either way.)
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT DISTINCT p
+ FROM products p, p.categories c
+ WHERE c = "Food";
+
+As another example, suppose that we want to find all orders that have *some* item with a large quantity.
+The following query will utilize the `oItemsQtyPriceIdx` we created, using only the first field in the index `qty`.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT DISTINCT o
+ FROM orders o, o.items i
+ WHERE i.qty > 100;
+
+
+## <a id="JoinQueries">Join Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+Finally, array indexes can also be used for index nested-loop joins if the field being joined is located within an array.
+Let us create another index on the `items` array of the `orders` dataset, this time on the `productno` field.
+
+ CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno);
+
+Now suppose we want to find all products located in a specific order.
+We can accomplish this with the join query below.
+Note that we must specify the `indexnl` join hint to tell AsterixDB that we want to optimize this specific join, as hash joins are the default join method otherwise.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT DISTINCT p
+ FROM products p, orders o, o.items i
+ WHERE i.productno /*+ indexnl */ = p.productno
+ AND o.custid = "C41";
+
+
+## <a id="ComplexIndexingExamples">Complex Indexing Examples</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
+
+### Open Indexes
+
+Similar to atomic indexes, array indexes are not limited to closed fields.
+The following DDLs illustrate how we could express `CREATE INDEX` statements comparable to those above if the to-be-indexed fields were not included in the their dataset's type definitions.
+
+ CREATE INDEX pCategoriesIdx ON products (UNNEST categories : string);
+ CREATE INDEX oItemsQtyPriceIdx ON orders (UNNEST items SELECT qty : int, price : int);
+ CREATE INDEX oProductIDIdx ON orders (UNNEST items SELECT productno : int);
+
+### Arrays in Arrays
+
+Array indexes are not just limited to arrays of depth = 1.
+We can generalize this to arrays of arbitrary depth, as long as an object encapsulates each array.
+The following DDLs describe indexing the `qty` field in an `items` array at various depths.
+
+ // { orderno: ..., items0: [ { items1: [ { qty: int, ... } ] } ] }
+ CREATE INDEX oItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 SELECT qty);
+
+ // { orderno: ..., items0: [ { items1: [ { items2: [ { qty: int, ... } ] } ] } ] }
+ CREATE INDEX oItemItemItemQtyIdx ON orders (UNNEST items0 UNNEST items1 UNNEST items2 SELECT qty);
+
+The queries below will utilize the indexes above.
+The first query utilizes the `oItemItemQtyIdx` index through nested existential quantification.
+The second query utilizes the `oItemItemItemQtyIdx` index with three unnesting clauses.
+
+ SET `compiler.arrayindex` "true";
+
+ SELECT o
+ FROM orders o
+ WHERE SOME o0 IN o.items0 SATISFIES (
+ SOME o1 IN o0.items1 SATISFIES o1.qty = 100
+ );
+
+ SELECT DISTINCT o
+ FROM orders o, o.items0 o0, o0.items1 o1, o1.items2 o2
+ WHERE o2.qty = 100;
diff --git a/asterixdb/asterix-doc/src/site/site.xml b/asterixdb/asterix-doc/src/site/site.xml
index ee71f9a..45193fc 100644
--- a/asterixdb/asterix-doc/src/site/site.xml
+++ b/asterixdb/asterix-doc/src/site/site.xml
@@ -97,6 +97,7 @@
<item name="Support of Full-text Queries" href="sqlpp/fulltext.html"/>
<item name="Support of Similarity Queries" href="sqlpp/similarity.html"/>
<item name="Support of Interval Joins" href="interval_join.html"/>
+ <item name="Support of Array Indexes" href="sqlpp/arrayindex.html"/>
</menu>
<menu name="Deprecated">
diff --git a/asterixdb/asterix-docker/pom.xml b/asterixdb/asterix-docker/pom.xml
index c14560c..df603c8 100644
--- a/asterixdb/asterix-docker/pom.xml
+++ b/asterixdb/asterix-docker/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-docker</artifactId>
diff --git a/asterixdb/asterix-examples/pom.xml b/asterixdb/asterix-examples/pom.xml
index 7c5c755..23efed4 100644
--- a/asterixdb/asterix-examples/pom.xml
+++ b/asterixdb/asterix-examples/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-examples</artifactId>
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 9676da6..9a7ef00 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarJavaFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarJavaFunctionEvaluator.java
index 35d55f7..4aa5f42 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarJavaFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarJavaFunctionEvaluator.java
@@ -55,9 +55,16 @@
String classname = finfo.getExternalIdentifier().get(0);
try {
- Class<?> clazz = Class.forName(classname, true, library.getClassLoader());
- IFunctionFactory externalFunctionFactory = (IFunctionFactory) clazz.newInstance();
- externalFunctionInstance = (IExternalScalarFunction) externalFunctionFactory.getExternalFunction();
+ //first, check if this class is assignable to the correct interface before running static initializers that
+ //may be dangerous
+ Class<?> clazz = Class.forName(classname, false, library.getClassLoader());
+ if (IFunctionFactory.class.isAssignableFrom(clazz)) {
+ //check if clazz implements IFunctionFactory
+ IFunctionFactory externalFunctionFactory = (IFunctionFactory) clazz.newInstance();
+ externalFunctionInstance = (IExternalScalarFunction) externalFunctionFactory.getExternalFunction();
+ } else {
+ throw new ClassCastException("Specified class does not implement IFunctionFactory");
+ }
} catch (Exception e) {
throw new RuntimeDataException(ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNABLE_TO_LOAD_CLASS, e, classname);
}
diff --git a/asterixdb/asterix-fuzzyjoin/pom.xml b/asterixdb/asterix-fuzzyjoin/pom.xml
index 5d8884a..66ae271d 100644
--- a/asterixdb/asterix-fuzzyjoin/pom.xml
+++ b/asterixdb/asterix-fuzzyjoin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-fuzzyjoin</artifactId>
@@ -125,6 +125,10 @@
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-dataflow-common</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-api</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
index 301d631..4c26fb3 100644
--- a/asterixdb/asterix-geo/pom.xml
+++ b/asterixdb/asterix-geo/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-geo</artifactId>
diff --git a/asterixdb/asterix-hivecompat/pom.xml b/asterixdb/asterix-hivecompat/pom.xml
index 25bdb8b..566ed9c 100644
--- a/asterixdb/asterix-hivecompat/pom.xml
+++ b/asterixdb/asterix-hivecompat/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index 041c098..cf2f741 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
index 1e3e68f..182fc08 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/Statement.java
@@ -68,7 +68,11 @@
WRITE,
CREATE_INDEX,
CREATE_DATAVERSE,
+ CREATE_FULL_TEXT_FILTER,
+ CREATE_FULL_TEXT_CONFIG,
INDEX_DROP,
+ FULL_TEXT_FILTER_DROP,
+ FULL_TEXT_CONFIG_DROP,
CREATE_FEED,
DROP_FEED,
START_FEED,
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java
new file mode 100644
index 0000000..2d304be
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextConfigStatement.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Iterator;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.FullTextUtil;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+public class CreateFullTextConfigStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+ private final String configName;
+ private final boolean ifNotExists;
+ private final AdmObjectNode configNode;
+
+ public static final String FIELD_NAME_TOKENIZER = "tokenizer";
+ public static final String FIELD_NAME_FILTER_PIPELINE = "filterPipeline";
+
+ public CreateFullTextConfigStatement(DataverseName dataverseName, String configName, boolean ifNotExists,
+ RecordConstructor expr) throws CompilationException {
+ this.dataverseName = dataverseName;
+ this.configName = configName;
+ this.ifNotExists = ifNotExists;
+ this.configNode = FullTextUtil.validateAndGetConfigNode(expr);
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getConfigName() {
+ return configName;
+ }
+
+ public boolean getIfNotExists() {
+ return ifNotExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_FULL_TEXT_CONFIG;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+
+ public TokenizerCategory getTokenizerCategory() throws HyracksDataException {
+ String tokenizerCategoryStr = configNode.getString(FIELD_NAME_TOKENIZER);
+ TokenizerCategory tokenizerCategory = TokenizerCategory.getEnumIgnoreCase(tokenizerCategoryStr);
+
+ return tokenizerCategory;
+ }
+
+ public ImmutableList<String> getFilterNames() {
+ AdmArrayNode arrayNode = (AdmArrayNode) configNode.get(FIELD_NAME_FILTER_PIPELINE);
+ ImmutableList.Builder<String> filterNamesBuilder = ImmutableList.builder();
+
+ Iterator<IAdmNode> iterator = arrayNode.iterator();
+ while (iterator.hasNext()) {
+ filterNamesBuilder.add(((AdmStringNode) iterator.next()).get());
+ }
+
+ return filterNamesBuilder.build();
+ }
+
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java
new file mode 100644
index 0000000..20427fd
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateFullTextFilterStatement.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import java.util.Iterator;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.util.FullTextUtil;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+import org.apache.asterix.object.base.AdmArrayNode;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.object.base.AdmStringNode;
+import org.apache.asterix.object.base.IAdmNode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.google.common.collect.ImmutableList;
+
+public class CreateFullTextFilterStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+ private final String filterName;
+ private final boolean ifNotExists;
+ private final AdmObjectNode filterNode;
+
+ public static final String FIELD_NAME_TYPE = "type";
+ public static final String FIELD_NAME_STOPWORDS_LIST = "stopwordsList";
+ public static final String FIELD_TYPE_STOPWORDS = "stopwords";
+
+ public CreateFullTextFilterStatement(DataverseName dataverseName, String filterName, boolean ifNotExists,
+ RecordConstructor expr) throws CompilationException {
+ this.dataverseName = dataverseName;
+ this.filterName = filterName;
+ this.ifNotExists = ifNotExists;
+ this.filterNode = FullTextUtil.getFilterNode(expr);
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getFilterName() {
+ return filterName;
+ }
+
+ public boolean getIfNotExists() {
+ return this.ifNotExists;
+ }
+
+ public String getFilterType() throws HyracksDataException {
+ return filterNode.getString(FIELD_NAME_TYPE);
+ }
+
+ public ImmutableList<String> getStopwordsList() throws CompilationException {
+ ImmutableList.Builder listBuiler = ImmutableList.<String> builder();
+ AdmArrayNode arrayNode = (AdmArrayNode) filterNode.get(FIELD_NAME_STOPWORDS_LIST);
+
+ Iterator<IAdmNode> iterator = arrayNode.iterator();
+ while (iterator.hasNext()) {
+ IAdmNode node = iterator.next();
+ if (!(node instanceof AdmStringNode)) {
+ throw new CompilationException(ErrorCode.PARSE_ERROR, getSourceLocation(),
+ "error when parsing stopwords list");
+ }
+
+ listBuiler.add(((AdmStringNode) node).get());
+ }
+
+ return listBuiler.build();
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.CREATE_FULL_TEXT_FILTER;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
index 05fc055..468006c 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/CreateIndexStatement.java
@@ -18,8 +18,10 @@
*/
package org.apache.asterix.lang.common.statement;
-import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -30,90 +32,67 @@
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
public class CreateIndexStatement extends AbstractStatement {
- private Identifier indexName;
- private DataverseName dataverseName;
- private Identifier datasetName;
- private List<Pair<List<String>, IndexedTypeExpression>> fieldExprs = new ArrayList<>();
- private List<Integer> fieldIndexIndicators = new ArrayList<>();
- private IndexType indexType = IndexType.BTREE;
- private boolean enforced;
- private boolean ifNotExists;
-
+ private final DataverseName dataverseName;
+ private final Identifier datasetName;
+ private final Identifier indexName;
+ private final IndexType indexType;
+ private final List<IndexedElement> indexedElements;
+ private final boolean enforced;
+ private final boolean ifNotExists;
// Specific to NGram indexes.
- private int gramLength = -1;
+ private final int gramLength;
+ // Specific to FullText indexes.
+ private final String fullTextConfigName;
- public CreateIndexStatement() {
- }
-
- public void setGramLength(int gramLength) {
+ public CreateIndexStatement(DataverseName dataverseName, Identifier datasetName, Identifier indexName,
+ IndexType indexType, List<IndexedElement> indexedElements, boolean enforced, int gramLength,
+ String fullTextConfigName, boolean ifNotExists) {
+ this.dataverseName = dataverseName;
+ this.datasetName = Objects.requireNonNull(datasetName);
+ this.indexName = Objects.requireNonNull(indexName);
+ this.indexType = Objects.requireNonNull(indexType);
+ this.indexedElements = Objects.requireNonNull(indexedElements);
+ this.enforced = enforced;
this.gramLength = gramLength;
+ this.ifNotExists = ifNotExists;
+ this.fullTextConfigName = fullTextConfigName;
}
- public int getGramLength() {
- return gramLength;
- }
-
- public Identifier getIndexName() {
- return indexName;
- }
-
- public void setIndexName(Identifier indexName) {
- this.indexName = indexName;
+ public String getFullTextConfigName() {
+ return fullTextConfigName;
}
public DataverseName getDataverseName() {
return dataverseName;
}
- public void setDataverseName(DataverseName dataverseName) {
- this.dataverseName = dataverseName;
- }
-
public Identifier getDatasetName() {
return datasetName;
}
- public void setDatasetName(Identifier datasetName) {
- this.datasetName = datasetName;
- }
-
- public List<Pair<List<String>, IndexedTypeExpression>> getFieldExprs() {
- return fieldExprs;
- }
-
- public void addFieldExprPair(Pair<List<String>, IndexedTypeExpression> fp) {
- this.fieldExprs.add(fp);
- }
-
- public List<Integer> getFieldSourceIndicators() {
- return fieldIndexIndicators;
- }
-
- public void addFieldIndexIndicator(Integer index) {
- fieldIndexIndicators.add(index);
+ public Identifier getIndexName() {
+ return indexName;
}
public IndexType getIndexType() {
return indexType;
}
- public void setIndexType(IndexType indexType) {
- this.indexType = indexType;
+ public List<IndexedElement> getIndexedElements() {
+ return indexedElements;
}
public boolean isEnforced() {
return enforced;
}
- public void setEnforced(boolean isEnforced) {
- this.enforced = isEnforced;
- }
-
- public void setIfNotExists(boolean ifNotExists) {
- this.ifNotExists = ifNotExists;
+ public int getGramLength() {
+ return gramLength;
}
public boolean getIfNotExists() {
@@ -125,17 +104,6 @@
return Statement.Kind.CREATE_INDEX;
}
- public boolean hasMetaField() {
- if (fieldIndexIndicators != null) {
- for (Integer indicator : fieldIndexIndicators) {
- if (indicator.intValue() != 0) {
- return true;
- }
- }
- }
- return false;
- }
-
@Override
public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
return visitor.visit(this, arg);
@@ -146,4 +114,57 @@
return Category.DDL;
}
+ public static final class IndexedElement {
+
+ private final int sourceIndicator;
+
+ private final List<List<String>> unnestList;
+
+ private final List<Pair<List<String>, IndexedTypeExpression>> projectList;
+
+ private SourceLocation sourceLoc;
+
+ public IndexedElement(int sourceIndicator, List<List<String>> unnestList,
+ List<Pair<List<String>, IndexedTypeExpression>> projectList) {
+ if (Objects.requireNonNull(projectList).isEmpty()) {
+ throw new IllegalArgumentException();
+ }
+ this.sourceIndicator = sourceIndicator;
+ this.unnestList = unnestList != null ? unnestList : Collections.emptyList();
+ this.projectList = projectList;
+ }
+
+ public int getSourceIndicator() {
+ return sourceIndicator;
+ }
+
+ public boolean hasUnnest() {
+ return !unnestList.isEmpty();
+ }
+
+ public List<List<String>> getUnnestList() {
+ return unnestList;
+ }
+
+ public List<Pair<List<String>, IndexedTypeExpression>> getProjectList() {
+ return projectList;
+ }
+
+ public Triple<Integer, List<List<String>>, List<List<String>>> toIdentifier() {
+ List<List<String>> newProjectList = projectList.stream().map(Pair::getFirst).collect(Collectors.toList());
+ return new Triple<>(sourceIndicator, unnestList, newProjectList);
+ }
+
+ public SourceLocation getSourceLocation() {
+ return sourceLoc;
+ }
+
+ public void setSourceLocation(SourceLocation sourceLoc) {
+ this.sourceLoc = sourceLoc;
+ }
+
+ public String getProjectListDisplayForm() {
+ return projectList.stream().map(Pair::getFirst).map(String::valueOf).collect(Collectors.joining(", "));
+ }
+ }
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java
new file mode 100644
index 0000000..ad84571
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextConfigDropStatement.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class FullTextConfigDropStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+ private final String configName;
+ private final boolean ifExists;
+
+ public FullTextConfigDropStatement(DataverseName dataverseName, String configName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.configName = configName;
+ this.ifExists = ifExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.FULL_TEXT_CONFIG_DROP;
+ }
+
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getConfigName() {
+ return configName;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java
new file mode 100644
index 0000000..9debca0
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/FullTextFilterDropStatement.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.lang.common.statement;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.lang.common.base.AbstractStatement;
+import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
+
+public class FullTextFilterDropStatement extends AbstractStatement {
+
+ private final DataverseName dataverseName;
+ private final String filterName;
+ private final boolean ifExists;
+
+ public FullTextFilterDropStatement(DataverseName dataverseName, String filterName, boolean ifExists) {
+ this.dataverseName = dataverseName;
+ this.filterName = filterName;
+ this.ifExists = ifExists;
+ }
+
+ @Override
+ public Kind getKind() {
+ return Kind.FULL_TEXT_FILTER_DROP;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ public String getFilterName() {
+ return filterName;
+ }
+
+ public boolean getIfExists() {
+ return ifExists;
+ }
+
+ @Override
+ public <R, T> R accept(ILangVisitor<R, T> visitor, T arg) throws CompilationException {
+ return visitor.visit(this, arg);
+ }
+
+ @Override
+ public byte getCategory() {
+ return Category.DDL;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java
new file mode 100644
index 0000000..c071dc9
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/FullTextUtil.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.lang.common.util;
+
+import static org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement.FIELD_NAME_FILTER_PIPELINE;
+import static org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement.FIELD_NAME_TOKENIZER;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.object.base.AdmObjectNode;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+public class FullTextUtil {
+
+ private FullTextUtil() {
+ }
+
+ //--------------------------------------- Full-text config --------------------------------------//
+
+ // Example of full-text config create statement
+ // CREATE FULLTEXT CONFIG my_second_stopword_config IF NOT EXISTS AS {
+ // "Tokenizer": "Word", // built-in tokenizers: "Word" or "NGram"
+ // "FilterPipeline": ["my_second_stopword_filter"]
+ // };
+ private static ARecordType getFullTextConfigRecordType() {
+ final String[] fieldNames = { FIELD_NAME_TOKENIZER, FIELD_NAME_FILTER_PIPELINE };
+ final IAType[] fieldTypes = { BuiltinType.ASTRING, new AOrderedListType(BuiltinType.ASTRING, null) };
+ return new ARecordType("fullTextConfigRecordType", fieldNames, fieldTypes, true);
+ }
+
+ private static final ARecordType FULL_TEXT_CONFIG_RECORD_TYPE = getFullTextConfigRecordType();
+
+ public static AdmObjectNode validateAndGetConfigNode(RecordConstructor recordConstructor)
+ throws CompilationException {
+ final ConfigurationTypeValidator validator = new ConfigurationTypeValidator();
+ final AdmObjectNode node = ExpressionUtils.toNode(recordConstructor);
+ validator.validateType(FULL_TEXT_CONFIG_RECORD_TYPE, node);
+ return node;
+ }
+
+ //--------------------------------------- Full-text filter --------------------------------------//
+
+ // Example of full-text filter create statement
+ // Note that only the type field is a must, and other fields is filter-type-specific
+ //
+ // CREATE FULLTEXT FILTER my_stopword_filter IF NOT EXISTS AS {
+ // "Type": "stopwords",
+ // "StopwordsList": ["xxx", "yyy", "zzz"]
+ // };
+
+ public static AdmObjectNode getFilterNode(RecordConstructor recordConstructor) throws CompilationException {
+ // Skip validation here because the current validator only supports CLOSED record validate
+ // while the FULL_TEXT_FILTER_RECORD_TYPE is open and specific to the filter types,
+ // e.g. stopwords filter and stemmer filter may have different fields
+ final AdmObjectNode node = ExpressionUtils.toNode(recordConstructor);
+ return node;
+ }
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 0474c0e..2650ca3 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -70,6 +70,8 @@
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -83,6 +85,8 @@
import org.apache.asterix.lang.common.statement.ExternalDetailsDecl;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -638,20 +642,52 @@
out.print(generateFullName(cis.getDataverseName(), cis.getDatasetName()));
out.print(" (");
- List<Pair<List<String>, IndexedTypeExpression>> fieldExprs = cis.getFieldExprs();
+ List<CreateIndexStatement.IndexedElement> indexedElements = cis.getIndexedElements();
int index = 0;
- int size = fieldExprs.size();
- for (Pair<List<String>, IndexedTypeExpression> entry : fieldExprs) {
- printNestField(entry.first);
- IndexedTypeExpression typeExpr = entry.second;
- if (typeExpr != null) {
- out.print(":");
- typeExpr.getType().accept(this, step);
- if (typeExpr.isUnknownable()) {
- out.print('?');
+ for (CreateIndexStatement.IndexedElement element : indexedElements) {
+ List<Pair<List<String>, IndexedTypeExpression>> projectList = element.getProjectList();
+
+ if (element.hasUnnest()) {
+ int innerIndex = 0;
+ out.print("(");
+ for (List<String> unnest : element.getUnnestList()) {
+ out.print(" unnest ");
+ printNestField(unnest);
+ if (++innerIndex < element.getUnnestList().size()) {
+ out.print(" ");
+ }
+ }
+
+ if (projectList.get(0).first != null) {
+ innerIndex = 0;
+ out.print(" select ");
+ for (Pair<List<String>, IndexedTypeExpression> project : projectList) {
+ printNestField(project.first);
+ if (project.second != null) {
+ out.print(":");
+ project.second.getType().accept(this, step);
+ if (project.second.isUnknownable()) {
+ out.print('?');
+ }
+ }
+ if (++innerIndex < element.getProjectList().size()) {
+ out.print(",");
+ }
+ }
+ }
+ out.print(")");
+ } else {
+ printNestField(projectList.get(0).first);
+ IndexedTypeExpression typeExpr = projectList.get(0).second;
+ if (typeExpr != null) {
+ out.print(":");
+ typeExpr.getType().accept(this, step);
+ if (typeExpr.isUnknownable()) {
+ out.print('?');
+ }
}
}
- if (++index < size) {
+ if (++index < indexedElements.size()) {
out.print(",");
}
}
@@ -686,6 +722,20 @@
}
@Override
+ public Void visit(CreateFullTextFilterStatement cis, Integer step) throws CompilationException {
+ out.print(skip(step) + "create fulltext filter " + cis.getFilterName());
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
+ public Void visit(CreateFullTextConfigStatement cis, Integer step) throws CompilationException {
+ out.print(skip(step) + "create fulltext config " + cis.getConfigName());
+ out.println(SEMICOLON);
+ return null;
+ }
+
+ @Override
public Void visit(IndexDropStatement del, Integer step) throws CompilationException {
out.print(skip(step) + "drop index ");
out.print(generateFullName(del.getDataverseName(), del.getDatasetName()));
@@ -719,6 +769,20 @@
}
@Override
+ public Void visit(FullTextFilterDropStatement del, Integer step) throws CompilationException {
+ out.print(skip(step) + "drop fulltext filter " + del.getFilterName());
+ out.println(generateIfExists(del.getIfExists()) + SEMICOLON);
+ return null;
+ }
+
+ @Override
+ public Void visit(FullTextConfigDropStatement del, Integer step) throws CompilationException {
+ out.print(skip(step) + "drop fulltext config " + del.getConfigName());
+ out.println(generateIfExists(del.getIfExists()) + SEMICOLON);
+ return null;
+ }
+
+ @Override
public Void visit(ConnectFeedStatement connectFeedStmt, Integer step) throws CompilationException {
out.print(skip(step) + "connect " + FEED);
out.print(generateFullName(connectFeedStmt.getDataverseName(), new Identifier(connectFeedStmt.getFeedName())));
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
index 9f4571d..0b4f56b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/AbstractQueryExpressionVisitor.java
@@ -31,6 +31,8 @@
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -43,6 +45,8 @@
import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -184,6 +188,26 @@
}
@Override
+ public R visit(CreateFullTextFilterStatement del, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(FullTextFilterDropStatement del, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(CreateFullTextConfigStatement del, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
+ public R visit(FullTextConfigDropStatement del, T arg) throws CompilationException {
+ return null;
+ }
+
+ @Override
public R visit(DisconnectFeedStatement del, T arg) throws CompilationException {
return null;
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
index 2957509..e36ec73 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/base/ILangVisitor.java
@@ -48,6 +48,8 @@
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
import org.apache.asterix.lang.common.statement.CreateFeedPolicyStatement;
import org.apache.asterix.lang.common.statement.CreateFeedStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateLibraryStatement;
@@ -60,6 +62,8 @@
import org.apache.asterix.lang.common.statement.DropDatasetStatement;
import org.apache.asterix.lang.common.statement.FeedDropStatement;
import org.apache.asterix.lang.common.statement.FeedPolicyDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
import org.apache.asterix.lang.common.statement.IndexDropStatement;
@@ -144,8 +148,16 @@
R visit(CreateDataverseStatement del, T arg) throws CompilationException;
+ R visit(CreateFullTextFilterStatement cis, T arg) throws CompilationException;
+
+ R visit(CreateFullTextConfigStatement cis, T arg) throws CompilationException;
+
R visit(IndexDropStatement del, T arg) throws CompilationException;
+ R visit(FullTextFilterDropStatement del, T arg) throws CompilationException;
+
+ R visit(FullTextConfigDropStatement del, T arg) throws CompilationException;
+
R visit(NodeGroupDropStatement del, T arg) throws CompilationException;
R visit(DataverseDropStatement del, T arg) throws CompilationException;
diff --git a/asterixdb/asterix-lang-sqlpp/pom.xml b/asterixdb/asterix-lang-sqlpp/pom.xml
index da49124..2cc1b4c 100644
--- a/asterixdb/asterix-lang-sqlpp/pom.xml
+++ b/asterixdb/asterix-lang-sqlpp/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 0aa9b58..c4c86f4 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -133,6 +133,8 @@
import org.apache.asterix.lang.common.statement.CreateFunctionStatement;
import org.apache.asterix.lang.common.statement.CreateIndexStatement;
import org.apache.asterix.lang.common.statement.CreateSynonymStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextFilterStatement;
+import org.apache.asterix.lang.common.statement.CreateFullTextConfigStatement;
import org.apache.asterix.lang.common.statement.DatasetDecl;
import org.apache.asterix.lang.common.statement.DataverseDecl;
import org.apache.asterix.lang.common.statement.DataverseDropStatement;
@@ -145,6 +147,8 @@
import org.apache.asterix.lang.common.statement.FunctionDecl;
import org.apache.asterix.lang.common.statement.FunctionDropStatement;
import org.apache.asterix.lang.common.statement.IndexDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextFilterDropStatement;
+import org.apache.asterix.lang.common.statement.FullTextConfigDropStatement;
import org.apache.asterix.lang.common.statement.InsertStatement;
import org.apache.asterix.lang.common.statement.InternalDetailsDecl;
import org.apache.asterix.lang.common.statement.LoadStatement;
@@ -224,6 +228,7 @@
private static final String GROUPS = "GROUPS";
private static final String IGNORE = "IGNORE";
private static final String LAST = "LAST";
+ private static final String META = "META";
private static final String NO = "NO";
private static final String NULLS = "NULLS";
private static final String OTHERS = "OTHERS";
@@ -239,6 +244,8 @@
private static final String UNBOUNDED = "UNBOUNDED";
private static final String REPLACE = "REPLACE";
private static final String RETURNS = "RETURNS";
+ private static final String CONFIG = "CONFIG";
+
private static final String INT_TYPE_NAME = "int";
private static final String UDF_VARARGS_PARAM_NAME = "args"; // Note: this value is stored in the function metadata
@@ -259,10 +266,12 @@
private static class IndexParams {
public IndexType type;
public int gramLength;
+ public String fullTextConfig;
- public IndexParams(IndexType type, int gramLength) {
+ public IndexParams(IndexType type, int gramLength, String fullTextConfig) {
this.type = type;
this.gramLength = gramLength;
+ this.fullTextConfig = fullTextConfig;
}
};
@@ -786,6 +795,7 @@
| stmt = CreateSynonymStatement(startToken)
| stmt = CreateFeedStatement(startToken)
| stmt = CreateFeedPolicyStatement(startToken)
+ | stmt = CreateFullTextStatement(startToken)
)
{
return stmt;
@@ -1088,50 +1098,168 @@
CreateIndexStatement IndexSpecification(Token startStmtToken) throws ParseException:
{
- CreateIndexStatement stmt = new CreateIndexStatement();
- String indexName = null;
- boolean ifNotExists = false;
Pair<DataverseName,Identifier> nameComponents = null;
- Pair<Integer, Pair<List<String>, IndexedTypeExpression>> fieldPair = null;
- IndexParams indexType = null;
+ String indexName = null;
+ IndexParams indexParams = null;
+ CreateIndexStatement.IndexedElement indexedElement = null;
+ List<CreateIndexStatement.IndexedElement> indexedElementList = new ArrayList<CreateIndexStatement.IndexedElement>();
boolean enforced = false;
+ boolean ifNotExists = false;
+ boolean hasUnnest = false;
+ String fullTextConfigName = null;
+ Token startElementToken = null;
}
{
(
indexName = Identifier() ifNotExists = IfNotExists()
<ON> nameComponents = QualifiedName()
- <LEFTPAREN> ( fieldPair = OpenField()
- {
- stmt.addFieldExprPair(fieldPair.second);
- stmt.addFieldIndexIndicator(fieldPair.first);
+ <LEFTPAREN> { startElementToken = token; }
+ indexedElement = IndexedElement(startElementToken) {
+ indexedElementList.add(indexedElement);
+ hasUnnest |= indexedElement.hasUnnest();
}
- ) (<COMMA> fieldPair = OpenField()
- {
- stmt.addFieldExprPair(fieldPair.second);
- stmt.addFieldIndexIndicator(fieldPair.first);
- }
- )* <RIGHTPAREN> ( <TYPE> indexType = IndexType() )? ( <ENFORCED> { enforced = true; } )?
+ (<COMMA> { startElementToken = token; }
+ indexedElement = IndexedElement(startElementToken) {
+ indexedElementList.add(indexedElement);
+ hasUnnest |= indexedElement.hasUnnest();
+ }
+ )*
+ <RIGHTPAREN>
+ ( <TYPE> indexParams = IndexType() )? ( <ENFORCED> { enforced = true; } )?
)
{
- stmt.setIndexName(new Identifier(indexName));
- stmt.setIfNotExists(ifNotExists);
- stmt.setDataverseName(nameComponents.first);
- stmt.setDatasetName(nameComponents.second);
- if (indexType != null) {
- stmt.setIndexType(indexType.type);
- stmt.setGramLength(indexType.gramLength);
+ IndexType indexType;
+ int gramLength;
+ if (indexParams != null) {
+ indexType = indexParams.type;
+ gramLength = indexParams.gramLength;
+ fullTextConfigName = indexParams.fullTextConfig;
+ } else {
+ indexType = hasUnnest ? IndexType.ARRAY : IndexType.BTREE;
+ gramLength = -1;
+ fullTextConfigName = null;
}
- stmt.setEnforced(enforced);
+ CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
+ new Identifier(indexName), indexType, indexedElementList, enforced, gramLength, fullTextConfigName, ifNotExists);
return addSourceLocation(stmt, startStmtToken);
}
}
+CreateIndexStatement.IndexedElement IndexedElement(Token startElementToken) throws ParseException:
+{
+ Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> element = null;
+ Pair<List<String>, IndexedTypeExpression> elementSimple = null;
+ int elementSimpleSource = 0;
+}
+{
+ (
+ element = IndexedElementUnnestSelect()
+ | (
+ LOOKAHEAD({ laIdentifier(META) && laToken(2, LEFTPAREN) && laToken(3, RIGHTPAREN) })
+ <IDENTIFIER> { expectToken(META); } <LEFTPAREN> <RIGHTPAREN>
+ <DOT> elementSimple = IndexedField()
+ { elementSimpleSource = 1; }
+ )
+ | elementSimple = IndexedField()
+ | <LEFTPAREN> ( element = IndexedElementUnnestSelect() | elementSimple = IndexedField() ) <RIGHTPAREN>
+ )
+ {
+ int source;
+ List<List<String>> unnestList;
+ List<Pair<List<String>, IndexedTypeExpression>> projectList;
+ if (elementSimple != null) {
+ source = elementSimpleSource;
+ unnestList = null;
+ projectList = Collections.singletonList(elementSimple);
+ } else {
+ source = element.first;
+ unnestList = element.second;
+ projectList = element.third;
+ }
+ CreateIndexStatement.IndexedElement ie = new CreateIndexStatement.IndexedElement(source, unnestList, projectList);
+ ie.setSourceLocation(getSourceLocation(startElementToken));
+ return ie;
+ }
+}
+
+Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> IndexedElementUnnestSelect()
+ throws ParseException:
+{
+ int source = 0;
+ Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> element = null;
+}
+{
+ <UNNEST>
+ (
+ (
+ LOOKAHEAD({ laIdentifier(META) && laToken(2, LEFTPAREN) && laToken(3, RIGHTPAREN) })
+ <IDENTIFIER> { expectToken(META); } <LEFTPAREN> <RIGHTPAREN>
+ <DOT> element = IndexedElementUnnestSelectBody() { source = 1; }
+ ) | element = IndexedElementUnnestSelectBody()
+ )
+ {
+ return new Triple<Integer, List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>>(
+ source, element.first, element.second
+ );
+ }
+}
+
+Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>> IndexedElementUnnestSelectBody()
+ throws ParseException:
+{
+ Triple<List<String>, Token, Token> path = null;
+ IndexedTypeExpression type = null;
+ List<List<String>> unnestList = new ArrayList();
+ List<Pair<List<String>, IndexedTypeExpression>> projectList = new ArrayList();
+}
+{
+ path = MultipartIdentifier() { unnestList.add(path.first); }
+ ( <UNNEST> path = MultipartIdentifier() { unnestList.add(path.first); })*
+ (
+ ( <COLON> type = IndexedTypeExpr(false)
+ {
+ projectList.add(new Pair<List<String>, IndexedTypeExpression>(null, type));
+ }
+ ) |
+ (
+ <SELECT> path = MultipartIdentifier() ( <COLON> type = IndexedTypeExpr(false) )?
+ {
+ projectList.add(new Pair<List<String>, IndexedTypeExpression>(path.first, type));
+ }
+ ( <COMMA> path = MultipartIdentifier() ( <COLON> type = IndexedTypeExpr(false) )?
+ {
+ projectList.add(new Pair<List<String>, IndexedTypeExpression>(path.first, type));
+ }
+ )*
+ )
+ )?
+ {
+ if (projectList.isEmpty()) {
+ // To support the case (<UNNEST> IDENTIFIER)* IDENTIFIER w/o any type specification.
+ projectList.add(new Pair<List<String>, IndexedTypeExpression>(null, null));
+ }
+
+ return new Pair<List<List<String>>, List<Pair<List<String>, IndexedTypeExpression>>>(unnestList, projectList);
+ }
+}
+
+Pair<List<String>, IndexedTypeExpression> IndexedField() throws ParseException:
+{
+ Triple<List<String>, Token, Token> path = null;
+ IndexedTypeExpression type = null;
+}
+{
+ path = MultipartIdentifier() ( <COLON> type = IndexedTypeExpr(true) )?
+ {
+ return new Pair<List<String>, IndexedTypeExpression>(path.first, type);
+ }
+}
+
CreateIndexStatement PrimaryIndexSpecification(Token startStmtToken) throws ParseException:
{
- CreateIndexStatement stmt = new CreateIndexStatement();
+ Pair<DataverseName,Identifier> nameComponents = null;
String indexName = null;
boolean ifNotExists = false;
- Pair<DataverseName,Identifier> nameComponents = null;
}
{
(indexName = Identifier())? ifNotExists = IfNotExists()
@@ -1140,10 +1268,8 @@
if (indexName == null) {
indexName = "primary_idx_" + nameComponents.second;
}
- stmt.setIndexName(new Identifier(indexName));
- stmt.setIfNotExists(ifNotExists);
- stmt.setDataverseName(nameComponents.first);
- stmt.setDatasetName(nameComponents.second);
+ CreateIndexStatement stmt = new CreateIndexStatement(nameComponents.first, nameComponents.second,
+ new Identifier(indexName), IndexType.BTREE, Collections.emptyList(), false, -1, null, ifNotExists);
return addSourceLocation(stmt, startStmtToken);
}
}
@@ -1163,6 +1289,7 @@
{
IndexType type = null;
int gramLength = 0;
+ String fullTextConfig = null;
}
{
(<BTREE>
@@ -1177,10 +1304,18 @@
{
type = IndexType.LENGTH_PARTITIONED_WORD_INVIX;
}
- |<FULLTEXT>
+ | <FULLTEXT>
{
type = IndexType.SINGLE_PARTITION_WORD_INVIX;
}
+ // For now we don't allow inverted index creation using a full-text config in another data verse.
+ // We may want to support corss-dataverse full-text config access later
+ // If so, replace the Identifier() with QualifiedName() to get the dataverse name
+ ( <USING> Identifier()
+ {
+ fullTextConfig = token.image;
+ }
+ )?
| <NGRAM> <LEFTPAREN> <INTEGER_LITERAL>
{
type = IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
@@ -1188,7 +1323,7 @@
}
<RIGHTPAREN>)
{
- return new IndexParams(type, gramLength);
+ return new IndexParams(type, gramLength, fullTextConfig);
}
}
@@ -1502,6 +1637,69 @@
}
}
+Statement CreateFullTextStatement(Token startStmtToken) throws ParseException:
+{
+ Statement stmt = null;
+}
+{
+ (
+ <FULLTEXT>
+ (
+ <FILTER> stmt = CreateFullTextFilterSpec(startStmtToken)
+ | (<IDENTIFIER> { expectToken(CONFIG); } stmt = CreateFullTextConfigSpec(startStmtToken))
+ )
+ )
+ {
+ return stmt;
+ }
+}
+
+CreateFullTextFilterStatement CreateFullTextFilterSpec(Token startStmtToken) throws ParseException:
+{
+ CreateFullTextFilterStatement stmt = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+ boolean ifNotExists = false;
+ RecordConstructor expr = null;
+}
+{
+ (
+ nameComponents = QualifiedName() ifNotExists = IfNotExists()
+ <AS>
+ expr = RecordConstructor()
+ )
+ {
+ try {
+ stmt = new CreateFullTextFilterStatement(nameComponents.first, nameComponents.second.getValue(), ifNotExists, expr);
+ return addSourceLocation(stmt, startStmtToken);
+ } catch (CompilationException e) {
+ throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
+ }
+ }
+}
+
+CreateFullTextConfigStatement CreateFullTextConfigSpec(Token startStmtToken) throws ParseException:
+{
+ CreateFullTextConfigStatement stmt = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+ boolean ifNotExists = false;
+ RecordConstructor expr = null;
+}
+{
+ (
+ nameComponents = QualifiedName() ifNotExists = IfNotExists()
+ <AS>
+ expr = RecordConstructor()
+ )
+ {
+ try {
+ stmt = new CreateFullTextConfigStatement(nameComponents.first, nameComponents.second.getValue(), ifNotExists, expr);
+ return addSourceLocation(stmt, startStmtToken);
+ } catch (CompilationException e) {
+ throw new SqlppParseException(getSourceLocation(startStmtToken), e.getMessage());
+ }
+ }
+}
+
CreateSynonymStatement CreateSynonymStatement(Token startStmtToken) throws ParseException:
{
CreateSynonymStatement stmt = null;
@@ -1653,6 +1851,7 @@
| stmt = DropFeedStatement(startToken)
| stmt = DropFeedPolicyStatement(startToken)
| stmt = DropSynonymStatement(startToken)
+ | stmt = DropFullTextStatement(startToken)
)
{
return stmt;
@@ -1707,6 +1906,50 @@
}
}
+Statement DropFullTextStatement(Token startStmtToken) throws ParseException:
+{
+ Statement stmt = null;
+}
+{
+ <FULLTEXT>
+ (
+ <FILTER> stmt = DropFullTextFilterSpec(startStmtToken)
+ | (<IDENTIFIER> { expectToken(CONFIG); } stmt = DropFullTextConfigSpec(startStmtToken))
+ )
+ {
+ return stmt;
+ }
+}
+
+FullTextFilterDropStatement DropFullTextFilterSpec(Token startStmtToken) throws ParseException:
+{
+ FullTextFilterDropStatement stmt = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+ boolean ifExists = false;
+}
+{
+ nameComponents = QualifiedName() ifExists = IfExists()
+ {
+ stmt = new FullTextFilterDropStatement(nameComponents.first, nameComponents.second.getValue(), ifExists);
+ return addSourceLocation(stmt, startStmtToken);
+ }
+}
+
+FullTextConfigDropStatement DropFullTextConfigSpec(Token startStmtToken) throws ParseException:
+{
+ FullTextConfigDropStatement stmt = null;
+ Pair<DataverseName,Identifier> nameComponents = null;
+ boolean ifExists = false;
+}
+{
+ nameComponents = QualifiedName() ifExists = IfExists()
+ {
+ stmt = new FullTextConfigDropStatement(nameComponents.first, nameComponents.second.getValue(), ifExists);
+ return addSourceLocation(stmt, startStmtToken);
+ }
+}
+
+
NodeGroupDropStatement DropNodeGroupStatement(Token startStmtToken) throws ParseException:
{
NodeGroupDropStatement stmt = null;
@@ -2306,14 +2549,21 @@
}
}
-IndexedTypeExpression IndexedTypeExpr() throws ParseException:
+IndexedTypeExpression IndexedTypeExpr(boolean allowQues) throws ParseException:
{
TypeExpression typeExpr = null;
- boolean isUnknownable = false;
+ boolean isUnknownable = !allowQues;
}
{
typeExpr = TypeExpr(false)
- ( <QUES> { isUnknownable = true; } )?
+ ( <QUES>
+ {
+ if (!allowQues) {
+ throw new SqlppParseException(getSourceLocation(token), "'?' quantifier is illegal for the type expression.");
+ }
+ isUnknownable = true;
+ }
+ )?
{
return new IndexedTypeExpression(typeExpr, isUnknownable);
}
@@ -2554,7 +2804,7 @@
}
{
fieldList = NestedField()
- ( <COLON> fieldType = IndexedTypeExpr() )?
+ ( <COLON> fieldType = IndexedTypeExpr(true) )?
{
return new Pair<Integer, Pair<List<String>, IndexedTypeExpression>>
(fieldList.first, new Pair<List<String>, IndexedTypeExpression>(fieldList.second, fieldType));
diff --git a/asterixdb/asterix-license/pom.xml b/asterixdb/asterix-license/pom.xml
index f59bb57..40d6050 100644
--- a/asterixdb/asterix-license/pom.xml
+++ b/asterixdb/asterix-license/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index 4bd413ab..c7d4a86 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>asterix-maven-plugins</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-grammar-extension-maven-plugin</artifactId>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
index 4bc481d..5e9a607 100644
--- a/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-test-datagenerator-maven-plugin/pom.xml
@@ -26,7 +26,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
index 40660a2..780ef82 100644
--- a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-maven-plugins/pom.xml b/asterixdb/asterix-maven-plugins/pom.xml
index 466f105..c7a463f 100644
--- a/asterixdb/asterix-maven-plugins/pom.xml
+++ b/asterixdb/asterix-maven-plugins/pom.xml
@@ -25,7 +25,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
index d13e5d3..540f49b 100644
--- a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>asterix-maven-plugins</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<packaging>maven-plugin</packaging>
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index 1e2bf53..de738a4 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-metadata</artifactId>
<properties>
@@ -178,7 +178,11 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs-client</artifactId>
</dependency>
- <dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-ipc</artifactId>
</dependency>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
index 116e55b..6602666 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataCache.java
@@ -37,12 +37,15 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.entities.Synonym;
import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
/**
* Caches metadata entities such that the MetadataManager does not have to
@@ -79,6 +82,10 @@
protected final Map<DataverseName, Map<String, FeedConnection>> feedConnections = new HashMap<>();
// Key is synonym dataverse. Key of value map is the synonym name
protected final Map<DataverseName, Map<String, Synonym>> synonyms = new HashMap<>();
+ // Key is DataverseName. Key of value map is the full-text filter name
+ protected final Map<DataverseName, Map<String, FullTextFilterMetadataEntity>> fullTextFilters = new HashMap<>();
+ // Key is DataverseName. Key of value map is the full-text config name
+ protected final Map<DataverseName, Map<String, FullTextConfigMetadataEntity>> fullTextConfigs = new HashMap<>();
// Atomically executes all metadata operations in ctx's log.
public void commit(MetadataTransactionContext ctx) {
@@ -113,20 +120,26 @@
synchronized (indexes) {
synchronized (datatypes) {
synchronized (functions) {
- synchronized (adapters) {
- synchronized (libraries) {
- synchronized (compactionPolicies) {
- synchronized (synonyms) {
- dataverses.clear();
- nodeGroups.clear();
- datasets.clear();
- indexes.clear();
- datatypes.clear();
- functions.clear();
- adapters.clear();
- libraries.clear();
- compactionPolicies.clear();
- synonyms.clear();
+ synchronized (fullTextConfigs) {
+ synchronized (fullTextFilters) {
+ synchronized (adapters) {
+ synchronized (libraries) {
+ synchronized (compactionPolicies) {
+ synchronized (synonyms) {
+ dataverses.clear();
+ nodeGroups.clear();
+ datasets.clear();
+ indexes.clear();
+ datatypes.clear();
+ functions.clear();
+ fullTextConfigs.clear();
+ fullTextFilters.clear();
+ adapters.clear();
+ libraries.clear();
+ compactionPolicies.clear();
+ synonyms.clear();
+ }
+ }
}
}
}
@@ -237,30 +250,37 @@
synchronized (indexes) {
synchronized (datatypes) {
synchronized (functions) {
- synchronized (adapters) {
- synchronized (libraries) {
- synchronized (feeds) {
- synchronized (compactionPolicies) {
- synchronized (synonyms) {
- datasets.remove(dataverse.getDataverseName());
- indexes.remove(dataverse.getDataverseName());
- datatypes.remove(dataverse.getDataverseName());
- adapters.remove(dataverse.getDataverseName());
- compactionPolicies.remove(dataverse.getDataverseName());
- List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<>();
- for (FunctionSignature signature : functions.keySet()) {
- if (signature.getDataverseName()
- .equals(dataverse.getDataverseName())) {
- markedFunctionsForRemoval.add(signature);
+ synchronized (fullTextConfigs) {
+ synchronized (fullTextFilters) {
+ synchronized (adapters) {
+ synchronized (libraries) {
+ synchronized (feeds) {
+ synchronized (compactionPolicies) {
+ synchronized (synonyms) {
+ datasets.remove(dataverse.getDataverseName());
+ indexes.remove(dataverse.getDataverseName());
+ datatypes.remove(dataverse.getDataverseName());
+ adapters.remove(dataverse.getDataverseName());
+ compactionPolicies.remove(dataverse.getDataverseName());
+ List<FunctionSignature> markedFunctionsForRemoval =
+ new ArrayList<>();
+ for (FunctionSignature signature : functions.keySet()) {
+ if (signature.getDataverseName()
+ .equals(dataverse.getDataverseName())) {
+ markedFunctionsForRemoval.add(signature);
+ }
+ }
+ for (FunctionSignature signature : markedFunctionsForRemoval) {
+ functions.remove(signature);
+ }
+ fullTextConfigs.remove(dataverse.getDataverseName());
+ fullTextFilters.remove(dataverse.getDataverseName());
+ libraries.remove(dataverse.getDataverseName());
+ feeds.remove(dataverse.getDataverseName());
+ synonyms.remove(dataverse.getDataverseName());
+ return dataverses.remove(dataverse.getDataverseName());
}
}
- for (FunctionSignature signature : markedFunctionsForRemoval) {
- functions.remove(signature);
- }
- libraries.remove(dataverse.getDataverseName());
- feeds.remove(dataverse.getDataverseName());
- synonyms.remove(dataverse.getDataverseName());
- return dataverses.remove(dataverse.getDataverseName());
}
}
}
@@ -376,6 +396,26 @@
}
}
+ public FullTextConfigMetadataEntity getFullTextConfig(DataverseName dataverseName, String configName) {
+ synchronized (fullTextConfigs) {
+ Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+ if (m == null) {
+ return null;
+ }
+ return m.get(configName);
+ }
+ }
+
+ public FullTextFilterMetadataEntity getFullTextFilter(DataverseName dataverseName, String filterName) {
+ synchronized (fullTextFilters) {
+ Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+ if (m == null) {
+ return null;
+ }
+ return m.get(filterName);
+ }
+ }
+
public List<Dataset> getDataverseDatasets(DataverseName dataverseName) {
synchronized (datasets) {
Map<String, Dataset> m = datasets.get(dataverseName);
@@ -436,6 +476,65 @@
}
}
+ public FullTextFilterMetadataEntity addFullTextFilterIfNotExists(FullTextFilterMetadataEntity filter) {
+ DataverseName dataverseName = filter.getFullTextFilter().getDataverseName();
+ String filterName = filter.getFullTextFilter().getName();
+ synchronized (fullTextFilters) {
+ Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+ if (m == null) {
+ m = new HashMap<>();
+ fullTextFilters.put(dataverseName, m);
+ }
+ if (!m.containsKey(filterName)) {
+ return m.put(filterName, filter);
+ }
+ return null;
+ }
+ }
+
+ public FullTextFilterMetadataEntity dropFullTextFilter(FullTextFilterMetadataEntity filterMetadataEntity) {
+ DataverseName dataverseName = filterMetadataEntity.getFullTextFilter().getDataverseName();
+ String filterName = filterMetadataEntity.getFullTextFilter().getName();
+ synchronized (fullTextFilters) {
+ Map<String, FullTextFilterMetadataEntity> m = fullTextFilters.get(dataverseName);
+ if (m == null) {
+ return null;
+ }
+ return m.remove(filterName);
+ }
+ }
+
+ public FullTextConfigMetadataEntity addFullTextConfigIfNotExists(
+ FullTextConfigMetadataEntity configMetadataEntity) {
+ FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+ DataverseName dataverseName = config.getDataverseName();
+ String configName = config.getName();
+ synchronized (fullTextConfigs) {
+ Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+ if (m == null) {
+ m = new HashMap<>();
+ fullTextConfigs.put(dataverseName, m);
+ }
+ if (!m.containsKey(configName)) {
+ return m.put(configName, configMetadataEntity);
+ }
+ return null;
+ }
+ }
+
+ public FullTextConfigMetadataEntity dropFullTextConfig(FullTextConfigMetadataEntity configMetadataEntity) {
+ FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+ DataverseName dataverseName = config.getDataverseName();
+ String configName = config.getName();
+ synchronized (fullTextConfigs) {
+ Map<String, FullTextConfigMetadataEntity> m = fullTextConfigs.get(dataverseName);
+ if (m == null) {
+ return null;
+ }
+ return m.remove(configName);
+ }
+ }
+
public Object addFeedPolicyIfNotExists(FeedPolicyEntity feedPolicy) {
synchronized (feedPolicies) {
Map<String, FeedPolicyEntity> p = feedPolicies.get(feedPolicy.getDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index 9a19972..0ab5c7b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -49,6 +49,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Library;
@@ -62,6 +64,8 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import com.google.common.base.Strings;
+
/**
* Provides access to Asterix metadata via remote methods to the metadata node.
* This metadata manager maintains a local cache of metadata Java objects
@@ -584,7 +588,7 @@
// requested function itself (but the function is still uncommitted).
Function function = ctx.getFunction(functionSignature);
if (function != null) {
- // Don't add this dataverse to the cache, since it is still
+ // Don't add this function to the cache, since it is still
// uncommitted.
return function;
}
@@ -627,6 +631,141 @@
}
@Override
+ public void addFullTextFilter(MetadataTransactionContext mdTxnCtx, FullTextFilterMetadataEntity filter)
+ throws AlgebricksException {
+ try {
+ metadataNode.addFullTextFilter(mdTxnCtx.getTxnId(), filter);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+ mdTxnCtx.addFullTextFilter(filter);
+ }
+
+ @Override
+ public void dropFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String filterName)
+ throws AlgebricksException {
+ try {
+ metadataNode.dropFullTextFilter(mdTxnCtx.getTxnId(), dataverseName, filterName);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+ mdTxnCtx.dropFullTextFilter(dataverseName, filterName);
+ }
+
+ @Override
+ public FullTextFilterMetadataEntity getFullTextFilter(MetadataTransactionContext ctx, DataverseName dataverseName,
+ String filterName) throws AlgebricksException {
+ // First look in the context to see if this transaction created the
+ // requested full-text filter itself (but the full-text filter is still uncommitted).
+ FullTextFilterMetadataEntity filter = ctx.getFullTextFilter(dataverseName, filterName);
+ if (filter != null) {
+ // Don't add this filter to the cache, since it is still
+ // uncommitted.
+ return filter;
+ }
+
+ if (ctx.fullTextFilterIsDropped(dataverseName, filterName)) {
+ // Filter 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;
+ }
+
+ filter = cache.getFullTextFilter(dataverseName, filterName);
+ if (filter != null) {
+ // filter is already in the cache, don't add it again.
+ return filter;
+ }
+
+ try {
+ filter = metadataNode.getFullTextFilter(ctx.getTxnId(), dataverseName, filterName);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+ // We fetched the filter from the MetadataNode. Add it to the cache
+ // when this transaction commits.
+ if (filter != null) {
+ ctx.addFullTextFilter(filter);
+ }
+ return filter;
+ }
+
+ @Override
+ public void addFullTextConfig(MetadataTransactionContext mdTxnCtx,
+ FullTextConfigMetadataEntity configMetadataEntity) throws AlgebricksException {
+ if (Strings.isNullOrEmpty(configMetadataEntity.getFullTextConfig().getName())) {
+ throw new MetadataException(ErrorCode.FULL_TEXT_CONFIG_ALREADY_EXISTS);
+ }
+
+ try {
+ metadataNode.addFullTextConfig(mdTxnCtx.getTxnId(), configMetadataEntity);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+ mdTxnCtx.addFullTextConfig(configMetadataEntity);
+ }
+
+ @Override
+ public FullTextConfigMetadataEntity getFullTextConfig(MetadataTransactionContext ctx, DataverseName dataverseName,
+ String configName) throws AlgebricksException {
+ // First look in the context to see if this transaction created the
+ // requested full-text config itself (but the full-text config is still uncommitted).
+ FullTextConfigMetadataEntity configMetadataEntity = ctx.getFullTextConfig(dataverseName, configName);
+ if (configMetadataEntity != null) {
+ // Don't add this config to the cache, since it is still
+ // uncommitted.
+ return configMetadataEntity;
+ }
+
+ if (ctx.fullTextConfigIsDropped(dataverseName, configName)) {
+ // config 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;
+ }
+
+ configMetadataEntity = cache.getFullTextConfig(dataverseName, configName);
+ if (configMetadataEntity != null) {
+ // config is already in the cache, don't add it again.
+ return configMetadataEntity;
+ }
+
+ try {
+ configMetadataEntity = metadataNode.getFullTextConfig(ctx.getTxnId(), dataverseName, configName);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+
+ // We fetched the config from the MetadataNode. Add it to the cache
+ // when this transaction commits.
+ if (configMetadataEntity != null) {
+ ctx.addFullTextConfig(configMetadataEntity);
+ }
+ return configMetadataEntity;
+ }
+
+ @Override
+ public void dropFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String configName)
+ throws AlgebricksException {
+ try {
+ metadataNode.dropFullTextConfig(mdTxnCtx.getTxnId(), dataverseName, configName);
+ } catch (RemoteException e) {
+ throw new MetadataException(ErrorCode.REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE, e);
+ }
+ mdTxnCtx.dropFullTextConfig(dataverseName, configName);
+ }
+
+ @Override
public void addFeedPolicy(MetadataTransactionContext mdTxnCtx, FeedPolicyEntity feedPolicy)
throws AlgebricksException {
try {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 20172db..f8ba574 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -20,6 +20,7 @@
package org.apache.asterix.metadata;
import static org.apache.asterix.common.api.IIdentifierMapper.Modifier.PLURAL;
+import static org.apache.asterix.common.exceptions.ErrorCode.FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED;
import static org.apache.asterix.common.utils.IdentifierUtil.dataset;
import java.rmi.RemoteException;
@@ -36,6 +37,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.dataflow.LSMIndexUtil;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
@@ -67,6 +69,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -83,6 +87,8 @@
import org.apache.asterix.metadata.entitytupletranslators.FeedConnectionTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.FeedPolicyTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.FeedTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.FullTextConfigMetadataEntityTupleTranslator;
+import org.apache.asterix.metadata.entitytupletranslators.FullTextFilterMetadataEntityTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
import org.apache.asterix.metadata.entitytupletranslators.LibraryTupleTranslator;
@@ -104,6 +110,7 @@
import org.apache.asterix.om.types.AbstractComplexType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
import org.apache.asterix.transaction.management.opcallbacks.UpsertOperationCallback;
@@ -136,6 +143,8 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import com.google.common.base.Strings;
+
public class MetadataNode implements IMetadataNode {
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = LogManager.getLogger();
@@ -451,6 +460,135 @@
}
}
+ @Override
+ public void addFullTextFilter(TxnId txnId, FullTextFilterMetadataEntity filterMetadataEntity)
+ throws RemoteException, AlgebricksException {
+ insertFullTextFilterMetadataEntityToCatalog(txnId, filterMetadataEntity);
+ return;
+ }
+
+ @Override
+ public FullTextFilterMetadataEntity getFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+ throws AlgebricksException {
+ try {
+ FullTextFilterMetadataEntityTupleTranslator translator =
+ tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+ ITupleReference searchKey = createTuple(dataverseName.getCanonicalForm(), filterName);
+ IValueExtractor<FullTextFilterMetadataEntity> valueExtractor =
+ new MetadataEntityValueExtractor<>(translator);
+ List<FullTextFilterMetadataEntity> results = new ArrayList<>();
+ searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, searchKey, valueExtractor, results);
+ if (results.isEmpty()) {
+ return null;
+ }
+ return results.get(0);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void dropFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+ throws AlgebricksException {
+ dropFullTextFilterDescriptor(txnId, dataverseName, filterName, false);
+ }
+
+ private void dropFullTextFilterDescriptor(TxnId txnId, DataverseName dataverseName, String filterName,
+ boolean force) throws AlgebricksException {
+ if (!force) {
+ confirmFullTextFilterCanBeDeleted(txnId, dataverseName, filterName);
+ }
+
+ try {
+ FullTextFilterMetadataEntityTupleTranslator translator =
+ tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+
+ ITupleReference key = createTuple(dataverseName.getCanonicalForm(), filterName);
+ deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, key);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ private void insertFullTextConfigMetadataEntityToCatalog(TxnId txnId, FullTextConfigMetadataEntity config)
+ throws AlgebricksException {
+ try {
+ FullTextConfigMetadataEntityTupleTranslator tupleReaderWriter =
+ tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+ ITupleReference configTuple = tupleReaderWriter.getTupleFromMetadataEntity(config);
+ insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, configTuple);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ private void insertFullTextFilterMetadataEntityToCatalog(TxnId txnId, FullTextFilterMetadataEntity filter)
+ throws AlgebricksException {
+ try {
+ FullTextFilterMetadataEntityTupleTranslator tupleReaderWriter =
+ tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+ ITupleReference filterTuple = tupleReaderWriter.getTupleFromMetadataEntity(filter);
+ insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, filterTuple);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
+ @Override
+ public void addFullTextConfig(TxnId txnId, FullTextConfigMetadataEntity config)
+ throws AlgebricksException, RemoteException {
+ try {
+ insertFullTextConfigMetadataEntityToCatalog(txnId, config);
+ } catch (AlgebricksException e) {
+ throw new AlgebricksException(e, ErrorCode.ERROR_PROCESSING_TUPLE);
+ }
+ }
+
+ @Override
+ public FullTextConfigMetadataEntity getFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+ throws AlgebricksException {
+ FullTextConfigMetadataEntityTupleTranslator translator =
+ tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+
+ ITupleReference searchKey;
+ List<FullTextConfigMetadataEntity> results = new ArrayList<>();
+ try {
+ searchKey = createTuple(dataverseName.getCanonicalForm(), configName);
+ IValueExtractor<FullTextConfigMetadataEntity> valueExtractor =
+ new MetadataEntityValueExtractor<>(translator);
+ searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, searchKey, valueExtractor, results);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+
+ if (results.isEmpty()) {
+ return null;
+ }
+
+ FullTextConfigMetadataEntity result = results.get(0);
+ return result;
+ }
+
+ @Override
+ public void dropFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+ throws AlgebricksException {
+ dropFullTextConfigDescriptor(txnId, dataverseName, configName, false);
+ }
+
+ private void dropFullTextConfigDescriptor(TxnId txnId, DataverseName dataverseName, String configName,
+ boolean force) throws AlgebricksException {
+ if (!force) {
+ confirmFullTextConfigCanBeDeleted(txnId, dataverseName, configName);
+ }
+
+ try {
+ ITupleReference key = createTuple(dataverseName.getCanonicalForm(), configName);
+ deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, key);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
private void insertTupleIntoIndex(TxnId txnId, IMetadataIndex metadataIndex, ITupleReference tuple)
throws HyracksDataException {
modifyMetadataIndex(Operation.INSERT, txnId, metadataIndex, tuple);
@@ -574,6 +712,25 @@
dropDataset(txnId, dataverseName, ds.getDatasetName(), true);
}
+ // Drop full-text configs in this dataverse.
+ // Note that full-text configs are utilized by the index, and we need to always drop index first
+ // and then full-text config
+ List<FullTextConfigMetadataEntity> configMetadataEntities =
+ getDataverseFullTextConfigs(txnId, dataverseName);
+ for (FullTextConfigMetadataEntity configMetadataEntity : configMetadataEntities) {
+ dropFullTextConfigDescriptor(txnId, dataverseName, configMetadataEntity.getFullTextConfig().getName(),
+ true);
+ }
+
+ // Drop full-text filters in this dataverse.
+ // Note that full-text filters are utilized by the full-text configs,
+ // and we need to always drop full-text configs first
+ // and then full-text filter
+ List<FullTextFilterMetadataEntity> filters = getDataverseFullTextFilters(txnId, dataverseName);
+ for (FullTextFilterMetadataEntity filter : filters) {
+ dropFullTextFilterDescriptor(txnId, dataverseName, filter.getFullTextFilter().getName(), true);
+ }
+
// Drop all types in this dataverse.
// As a side effect, acquires an S lock on the 'datatype' dataset on behalf of txnId.
List<Datatype> dataverseDatatypes = getDataverseDatatypes(txnId, dataverseName);
@@ -605,7 +762,9 @@
|| !getDataverseFunctions(txnId, dataverseName).isEmpty()
|| !getDataverseFeedPolicies(txnId, dataverseName).isEmpty()
|| !getDataverseFeeds(txnId, dataverseName).isEmpty()
- || !getDataverseSynonyms(txnId, dataverseName).isEmpty();
+ || !getDataverseSynonyms(txnId, dataverseName).isEmpty()
+ || !getDataverseFullTextConfigs(txnId, dataverseName).isEmpty()
+ || !getDataverseFullTextFilters(txnId, dataverseName).isEmpty();
}
@Override
@@ -854,6 +1013,38 @@
}
}
+ private List<FullTextConfigMetadataEntity> getDataverseFullTextConfigs(TxnId txnId, DataverseName dataverseName)
+ throws AlgebricksException {
+ ITupleReference searchKey = createTuple(dataverseName);
+ FullTextConfigMetadataEntityTupleTranslator tupleReaderWriter =
+ tupleTranslatorProvider.getFullTextConfigTupleTranslator(true);
+ IValueExtractor<FullTextConfigMetadataEntity> valueExtractor =
+ new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<FullTextConfigMetadataEntity> results = new ArrayList<>();
+ try {
+ searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, searchKey, valueExtractor, results);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return results;
+ }
+
+ private List<FullTextFilterMetadataEntity> getDataverseFullTextFilters(TxnId txnId, DataverseName dataverseName)
+ throws AlgebricksException {
+ ITupleReference searchKey = createTuple(dataverseName);
+ FullTextFilterMetadataEntityTupleTranslator tupleReaderWriter =
+ tupleTranslatorProvider.getFullTextFilterTupleTranslator(true);
+ IValueExtractor<FullTextFilterMetadataEntity> valueExtractor =
+ new MetadataEntityValueExtractor<>(tupleReaderWriter);
+ List<FullTextFilterMetadataEntity> results = new ArrayList<>();
+ try {
+ searchIndex(txnId, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, searchKey, valueExtractor, results);
+ } catch (HyracksDataException e) {
+ throw new AlgebricksException(e);
+ }
+ return results;
+ }
+
@Override
public Dataset getDataset(TxnId txnId, DataverseName dataverseName, String datasetName) throws AlgebricksException {
try {
@@ -1039,6 +1230,37 @@
}
}
+ private void confirmFullTextConfigCanBeDeleted(TxnId txnId, DataverseName dataverseNameFullTextConfig,
+ String configName) throws AlgebricksException {
+ if (Strings.isNullOrEmpty(configName)) {
+ throw new MetadataException(FULL_TEXT_DEFAULT_CONFIG_CANNOT_BE_DELETED_OR_CREATED);
+ }
+
+ // If any index uses this full-text config, throw an error
+ List<Dataset> datasets = getAllDatasets(txnId);
+ for (Dataset dataset : datasets) {
+ List<Index> indexes = getDatasetIndexes(txnId, dataset.getDataverseName(), dataset.getDatasetName());
+ for (Index index : indexes) {
+ // ToDo: to support index to access full-text config in another dataverse,
+ // we may need to include the dataverse of the full-text config in the index.getFullTextConfigDataverse()
+ // and instead of checking index.getDataverseName(), we need to check index.getFullTextConfigDataverse()
+ // to see if it is the same as the dataverse of the full-text config
+ if (Index.IndexCategory.of(index.getIndexType()) == Index.IndexCategory.TEXT) {
+ String indexConfigName = ((Index.TextIndexDetails) index.getIndexDetails()).getFullTextConfigName();
+ if (index.getDataverseName().equals(dataverseNameFullTextConfig)
+ && !Strings.isNullOrEmpty(indexConfigName) && indexConfigName.equals(configName)) {
+ throw new AsterixException(
+ org.apache.asterix.common.exceptions.ErrorCode.CANNOT_DROP_OBJECT_DEPENDENT_EXISTS,
+ "full-text config",
+ MetadataUtil.getFullyQualifiedDisplayName(dataverseNameFullTextConfig, configName),
+ "index", DatasetUtil.getFullyQualifiedDisplayName(index.getDataverseName(),
+ index.getDatasetName()) + "." + index.getIndexName());
+ }
+ }
+ }
+ }
+ }
+
private void confirmDatasetCanBeDeleted(TxnId txnId, DataverseName dataverseName, String datasetName)
throws AlgebricksException {
confirmDatasetIsUnusedByFunctions(txnId, dataverseName, datasetName);
@@ -1137,6 +1359,22 @@
null);
}
+ private void confirmFullTextFilterCanBeDeleted(TxnId txnId, DataverseName dataverseName, String fullTextFilterName)
+ throws AlgebricksException {
+ List<FullTextConfigMetadataEntity> configMetadataEntities = getDataverseFullTextConfigs(txnId, dataverseName);
+ for (FullTextConfigMetadataEntity configMetadataEntity : configMetadataEntities) {
+ FullTextConfigDescriptor config = configMetadataEntity.getFullTextConfig();
+ for (String filterName : config.getFilterNames()) {
+ if (filterName.equals(fullTextFilterName)) {
+ throw new AlgebricksException("Cannot drop full-text filter "
+ + TypeUtil.getFullyQualifiedDisplayName(dataverseName, fullTextFilterName)
+ + " being used by full-text config "
+ + TypeUtil.getFullyQualifiedDisplayName(dataverseName, config.getName()));
+ }
+ }
+ }
+ }
+
private List<String> getNestedComplexDatatypeNamesForThisDatatype(TxnId txnId, DataverseName dataverseName,
String datatypeName) throws AlgebricksException {
// Return all field types that aren't builtin types
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
index 8da01aa..a6b1f4b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataTransactionContext.java
@@ -34,11 +34,17 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Library;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.utils.MetadataUtil;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
/**
* Used to implement serializable transactions against the MetadataCache.
@@ -110,6 +116,16 @@
logAndApply(new MetadataLogicalOperation(function, true));
}
+ public void addFullTextFilter(FullTextFilterMetadataEntity filterMetadataEntity) {
+ droppedCache.dropFullTextFilter(filterMetadataEntity);
+ logAndApply(new MetadataLogicalOperation(filterMetadataEntity, true));
+ }
+
+ public void addFullTextConfig(FullTextConfigMetadataEntity configMetadataEntity) {
+ droppedCache.dropFullTextConfig(configMetadataEntity);
+ logAndApply(new MetadataLogicalOperation(configMetadataEntity, true));
+ }
+
public void addAdapter(DatasourceAdapter adapter) {
droppedCache.dropAdapterIfExists(adapter);
logAndApply(new MetadataLogicalOperation(adapter, true));
@@ -128,8 +144,8 @@
}
public void dropIndex(DataverseName dataverseName, String datasetName, String indexName) {
- Index index = new Index(dataverseName, datasetName, indexName, null, null, null, null, false, false, false,
- MetadataUtil.PENDING_NO_OP);
+ Index index =
+ new Index(dataverseName, datasetName, indexName, null, null, false, false, MetadataUtil.PENDING_NO_OP);
droppedCache.addIndexIfNotExists(index);
logAndApply(new MetadataLogicalOperation(index, false));
}
@@ -164,6 +180,31 @@
logAndApply(new MetadataLogicalOperation(function, false));
}
+ public void dropFullTextConfig(DataverseName dataverseName, String configName) {
+ FullTextConfigDescriptor config = new FullTextConfigDescriptor(dataverseName, configName, null, null);
+ FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(config);
+
+ droppedCache.addFullTextConfigIfNotExists(configMetadataEntity);
+ logAndApply(new MetadataLogicalOperation(configMetadataEntity, false));
+ }
+
+ public void dropFullTextFilter(DataverseName dataverseName, String filterName) {
+ AbstractFullTextFilterDescriptor filter = new AbstractFullTextFilterDescriptor(dataverseName, filterName) {
+ @Override
+ public FullTextFilterType getFilterType() {
+ return null;
+ }
+
+ @Override
+ public IFullTextFilterEvaluatorFactory createEvaluatorFactory() {
+ return null;
+ }
+ };
+ FullTextFilterMetadataEntity filterMetadataEntity = new FullTextFilterMetadataEntity(filter);
+ droppedCache.addFullTextFilterIfNotExists(filterMetadataEntity);
+ logAndApply(new MetadataLogicalOperation(filterMetadataEntity, false));
+ }
+
public void dropAdapter(DataverseName dataverseName, String adapterName) {
AdapterIdentifier adapterIdentifier = new AdapterIdentifier(dataverseName, adapterName);
DatasourceAdapter adapter = new DatasourceAdapter(adapterIdentifier, null, null, null, null);
@@ -218,6 +259,14 @@
return droppedCache.getFunction(functionSignature) != null;
}
+ public boolean fullTextConfigIsDropped(DataverseName dataverseName, String configName) {
+ return droppedCache.getFullTextConfig(dataverseName, configName) != null;
+ }
+
+ public boolean fullTextFilterIsDropped(DataverseName dataverseName, String filterName) {
+ return droppedCache.getFullTextFilter(dataverseName, filterName) != null;
+ }
+
public List<MetadataLogicalOperation> getOpLog() {
return opLog;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
index 139216e..5ff8a03 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataManager.java
@@ -36,6 +36,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Library;
@@ -422,6 +424,81 @@
/**
* @param mdTxnCtx
* MetadataTransactionContext of an active metadata transaction.
+ * @param filterMetadataEntity
+ * the full-text filter descriptor to be added
+ * @throws AlgebricksException
+ * For example, if the filter with the same name in the same dataverse already exists
+ */
+ void addFullTextFilter(MetadataTransactionContext mdTxnCtx, FullTextFilterMetadataEntity filterMetadataEntity)
+ throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param filterName
+ * the name of the full-text filter to be fetched
+ * @throws AlgebricksException
+ * For example, if the filter doesn't exist
+ */
+ FullTextFilterMetadataEntity getFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+ String filterName) throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param filterName
+ * the name of the full-text filter to be dropped
+ * @throws AlgebricksException
+ * For example, if ifExists is set to false and the filter doesn't exist
+ */
+ void dropFullTextFilter(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String filterName)
+ throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param configMetadataEntity
+ * the full-text config descriptor to be added
+ * @throws AlgebricksException
+ * For example, if the config with the same name in the same dataverse already exists
+ */
+ void addFullTextConfig(MetadataTransactionContext mdTxnCtx, FullTextConfigMetadataEntity configMetadataEntity)
+ throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param configName
+ * the name of the full-text config to be fetched
+ * @throws AlgebricksException
+ * For example, if the full-text config doesn't exist
+ * @return
+ */
+ FullTextConfigMetadataEntity getFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
+ String configName) throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param configName
+ * the name of the full-text config to be dropped
+ * @throws AlgebricksException
+ * For example, if ifExists is set to false and the config doesn't exist
+ */
+ void dropFullTextConfig(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName, String configName)
+ throws AlgebricksException;
+
+ /**
+ * @param mdTxnCtx
+ * MetadataTransactionContext of an active metadata transaction.
* @param adapter
* An instance of type Adapter that represents the adapter being
* added
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 08ed0a3..561a4fa 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -36,6 +36,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Library;
@@ -449,6 +451,80 @@
/**
* @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param filterMetadataEntity
+ * the full-text filter metadata entity to be added
+ * @throws AlgebricksException
+ * For example, if the filter with the same name in the same dataverse already exists
+ */
+ void addFullTextFilter(TxnId txnId, FullTextFilterMetadataEntity filterMetadataEntity)
+ throws RemoteException, AlgebricksException;
+
+ /**
+ * @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param filterName
+ * the name of the full-text filter to be fetched
+ * @throws AlgebricksException
+ * For example, if the filter doesn't exist
+ */
+ FullTextFilterMetadataEntity getFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+ throws RemoteException, AlgebricksException;
+
+ /**
+ * @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param filterName
+ * the name of the full-text filter to be dropped
+ * @throws AlgebricksException
+ * For example, if ifExists is set to false and the filter doesn't exist
+ */
+ void dropFullTextFilter(TxnId txnId, DataverseName dataverseName, String filterName)
+ throws RemoteException, AlgebricksException;
+
+ /**
+ * @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param configMetadataEntity
+ * the full-text config descriptor to be added
+ * @throws AlgebricksException
+ * For example, if the config with the same name in the same dataverse already exists
+ */
+ void addFullTextConfig(TxnId txnId, FullTextConfigMetadataEntity configMetadataEntity)
+ throws AlgebricksException, RemoteException;
+
+ /**
+ * @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param configName
+ * the name of the full-text config to be fetched
+ * @throws AlgebricksException
+ * For example, if the full-text config doesn't exist
+ */
+ FullTextConfigMetadataEntity getFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+ throws AlgebricksException, RemoteException;
+
+ /**
+ * @param txnId
+ * Metadata transaction id of an active metadata transaction.
+ * @param dataverseName
+ * the name of the dataverse where the full-text filter belongs
+ * @param configName
+ * the name of the full-text config to be dropped
+ * @throws AlgebricksException
+ * For example, if ifExists is set to false and the config doesn't exist
+ */
+ void dropFullTextConfig(TxnId txnId, DataverseName dataverseName, String configName)
+ throws RemoteException, AlgebricksException;
+
+ /**
+ * @param txnId
* @param dataverseName
* @return List<Adapter> A list containing the adapters in the specified
* dataverse
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 8cfd146..047b823 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -123,7 +123,8 @@
MetadataPrimaryIndexes.FEED_DATASET, MetadataPrimaryIndexes.FEED_POLICY_DATASET,
MetadataPrimaryIndexes.LIBRARY_DATASET, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET,
MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, MetadataPrimaryIndexes.FEED_CONNECTION_DATASET,
- MetadataPrimaryIndexes.SYNONYM_DATASET };
+ MetadataPrimaryIndexes.SYNONYM_DATASET, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET,
+ MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET };
private MetadataBootstrap() {
}
@@ -173,6 +174,7 @@
} else {
insertNewCompactionPoliciesIfNotExist(mdTxnCtx);
insertSynonymEntitiesIfNotExist(mdTxnCtx);
+ insertFullTextConfigAndFilterIfNotExist(mdTxnCtx);
}
// #. initialize datasetIdFactory
MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
@@ -314,6 +316,38 @@
}
}
+ // For backward-compatibility: for old datasets created by an older version of AsterixDB, they
+ // 1) may not have such a full-text config dataset in the metadata catalog,
+ // 2) may not have the default full-text config as an entry in the metadata catalog
+ // So here, let's try to insert if not exists
+ private static void insertFullTextConfigAndFilterIfNotExist(MetadataTransactionContext mdTxnCtx)
+ throws AlgebricksException {
+
+ // We need to insert data types first because datasets depend on data types
+ // ToDo: create a new function to reduce duplicated code here: addDatatypeIfNotExist()
+ IAType fullTextConfigRecordType = MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET.getPayloadRecordType();
+ if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ fullTextConfigRecordType.getTypeName()) == null) {
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+ fullTextConfigRecordType.getTypeName(), fullTextConfigRecordType, false));
+ }
+ IAType fullTextFilterRecordType = MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET.getPayloadRecordType();
+ if (MetadataManager.INSTANCE.getDatatype(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ fullTextFilterRecordType.getTypeName()) == null) {
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(MetadataConstants.METADATA_DATAVERSE_NAME,
+ fullTextFilterRecordType.getTypeName(), fullTextFilterRecordType, false));
+ }
+
+ if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME) == null) {
+ insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET });
+ }
+ if (MetadataManager.INSTANCE.getDataset(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+ MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME) == null) {
+ insertMetadataDatasets(mdTxnCtx, new IMetadataIndex[] { MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET });
+ }
+ }
+
private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws AlgebricksException {
try {
String adapterName =
@@ -528,7 +562,11 @@
}
private static void ensureCatalogUpgradability(IMetadataIndex index) {
- if (index != MetadataPrimaryIndexes.SYNONYM_DATASET) {
+ if (index != MetadataPrimaryIndexes.SYNONYM_DATASET
+ // Backward-compatibility: FULLTEXT_ENTITY_DATASET is added to AsterixDB recently
+ // and may not exist in an older dataverse
+ && index != MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET
+ && index != MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET) {
throw new IllegalStateException(
"attempt to create metadata index " + index.getIndexName() + ". Index should already exist");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 556090a..a191e43 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -61,6 +61,10 @@
new MetadataIndexImmutableProperties(MetadataConstants.EXTERNAL_FILE_DATASET_NAME, 14, 14);
public static final MetadataIndexImmutableProperties PROPERTIES_SYNONYM =
new MetadataIndexImmutableProperties(MetadataConstants.SYNONYM_DATASET_NAME, 15, 15);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_CONFIG =
+ new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_CONFIG_DATASET_NAME, 16, 16);
+ public static final MetadataIndexImmutableProperties PROPERTIES_FULL_TEXT_FILTER =
+ new MetadataIndexImmutableProperties(MetadataConstants.FULL_TEXT_FILTER_DATASET_NAME, 17, 17);
public static final IMetadataIndex DATAVERSE_DATASET =
new MetadataIndex(PROPERTIES_DATAVERSE, 2, new IAType[] { BuiltinType.ASTRING },
@@ -142,6 +146,17 @@
Arrays.asList(MetadataRecordTypes.FIELD_NAME_SYNONYM_NAME)),
0, MetadataRecordTypes.SYNONYM_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex FULL_TEXT_CONFIG_DATASET =
+ new MetadataIndex(PROPERTIES_FULL_TEXT_CONFIG, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_CONFIG_NAME)),
+ 0, MetadataRecordTypes.FULL_TEXT_CONFIG_RECORDTYPE, true, new int[] { 0, 1 });
+ public static final IMetadataIndex FULL_TEXT_FILTER_DATASET =
+ new MetadataIndex(PROPERTIES_FULL_TEXT_FILTER, 3, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING },
+ Arrays.asList(Arrays.asList(MetadataRecordTypes.FIELD_NAME_DATAVERSE_NAME),
+ Arrays.asList(MetadataRecordTypes.FIELD_NAME_FULL_TEXT_FILTER_NAME)),
+ 0, MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE, true, new int[] { 0, 1 });
+
private MetadataPrimaryIndexes() {
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 10f5047..2e75319 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -110,6 +110,13 @@
public static final String FIELD_NAME_WORKING_MEMORY_SIZE = "WorkingMemorySize";
public static final String FIELD_NAME_APPLIED_FUNCTIONS = "AppliedFunctions";
public static final String FIELD_NAME_WHERE_CLAUSE = "WhereClause";
+ public static final String FIELD_NAME_FULL_TEXT_CONFIG_NAME = "FullTextConfigName";
+ public static final String FIELD_NAME_FULL_TEXT_FILTER_NAME = "FullTextFilterName";
+ public static final String FIELD_NAME_FULL_TEXT_FILTER_TYPE = "FullTextFilterType";
+ public static final String FIELD_NAME_FULL_TEXT_TOKENIZER = "Tokenizer";
+ public static final String FIELD_NAME_FULL_TEXT_FILTER_PIPELINE = "FullTextFilterPipeline";
+ public static final String FIELD_NAME_FULL_TEXT_STOPWORD_LIST = "StopwordList";
+ public static final String FIELD_NAME_FULL_TEXT_STEMMER_LANGUAGE = "Language";
//---------------------------------- Record Types Creation ----------------------------------//
//--------------------------------------- Properties ----------------------------------------//
@@ -510,6 +517,41 @@
//IsOpen?
true);
+ //---------------------------------- FullText Config and Filter -------------------------//
+ public static final int FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX = 0;
+
+ // FullText Config
+ public static final int FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX = 1;
+ public static final int FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX = 2;
+ public static final int FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX = 3;
+
+ // FullText Filter
+ public static final int FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX = 1;
+ public static final int FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX = 2;
+
+ // Stopword Filter
+ public static final int FULLTEXT_ENTITY_ARECORD_STOPWORD_LIST_FIELD_INDEX = 3;
+
+ // Stemmer Filter
+ public static final int FULLTEXT_ENTITY_ARECORD_STEMMER_LANGUAGE_FIELD_INDEX = 3;
+
+ public static final String RECORD_NAME_FULL_TEXT_CONFIG = "FullTextConfigRecordType";
+ public static final ARecordType FULL_TEXT_CONFIG_RECORDTYPE = createRecordType(RECORD_NAME_FULL_TEXT_CONFIG,
+ new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FULL_TEXT_CONFIG_NAME, FIELD_NAME_FULL_TEXT_TOKENIZER,
+ FIELD_NAME_FULL_TEXT_FILTER_PIPELINE },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, AUnionType.createNullableType(BuiltinType.ASTRING),
+ AUnionType
+ .createNullableType(new AOrderedListType(BuiltinType.ASTRING, "FullTextFilterPipeline")) },
+ true);
+
+ public static final String RECORD_NAME_FULL_TEXT_FILTER = "FullTextFilterRecordType";
+ // Different filters may have different fields, e.g.
+ // stopwords filter has the stopwords list, so this type is OPEN
+ public static final ARecordType FULL_TEXT_FILTER_RECORDTYPE = createRecordType(RECORD_NAME_FULL_TEXT_FILTER,
+ new String[] { FIELD_NAME_DATAVERSE_NAME, FIELD_NAME_FULL_TEXT_FILTER_NAME,
+ FIELD_NAME_FULL_TEXT_FILTER_TYPE },
+ new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, true);
+
// private members
private MetadataRecordTypes() {
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
new file mode 100644
index 0000000..7c19282
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/ArrayBTreeResourceFactoryProvider.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.metadata.declared;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.api.IResourceFactoryProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.metadata.utils.IndexUtil;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.api.compression.ICompressorDecompressorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
+import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
+import org.apache.hyracks.storage.common.IResourceFactory;
+import org.apache.hyracks.storage.common.IStorageManager;
+import org.apache.hyracks.storage.common.compression.NoOpCompressorDecompressorFactory;
+
+public class ArrayBTreeResourceFactoryProvider implements IResourceFactoryProvider {
+
+ public static final ArrayBTreeResourceFactoryProvider INSTANCE = new ArrayBTreeResourceFactoryProvider();
+
+ private ArrayBTreeResourceFactoryProvider() {
+ }
+
+ @Override
+ public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Dataset dataset, Index index,
+ ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
+ Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
+ IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
+ int[] filterFields = IndexUtil.getFilterFields(dataset, index, filterTypeTraits);
+ int[] btreeFields = IndexUtil.getBtreeFieldsIfFiltered(dataset, index);
+ IStorageComponentProvider storageComponentProvider = mdProvider.getStorageComponentProvider();
+ ITypeTraits[] typeTraits = getTypeTraits(mdProvider, dataset, index, recordType, metaType);
+ IBinaryComparatorFactory[] cmpFactories = getCmpFactories(mdProvider, dataset, index, recordType, metaType);
+ double bloomFilterFalsePositiveRate = mdProvider.getStorageProperties().getBloomFilterFalsePositiveRate();
+ ILSMOperationTrackerFactory opTrackerFactory = dataset.getIndexOperationTrackerFactory(index);
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory = dataset.getIoOperationCallbackFactory(index);
+ ILSMPageWriteCallbackFactory pageWriteCallbackFactory = dataset.getPageWriteCallbackFactory();
+ IStorageManager storageManager = storageComponentProvider.getStorageManager();
+ IMetadataPageManagerFactory metadataPageManagerFactory =
+ storageComponentProvider.getMetadataPageManagerFactory();
+ ILSMIOOperationSchedulerProvider ioSchedulerProvider =
+ storageComponentProvider.getIoOperationSchedulerProvider();
+ switch (dataset.getDatasetType()) {
+ case EXTERNAL:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes are not " + "supported for external datasets.");
+ case INTERNAL:
+ AsterixVirtualBufferCacheProvider vbcProvider =
+ new AsterixVirtualBufferCacheProvider(dataset.getDatasetId());
+
+ final ICompressorDecompressorFactory compDecompFactory;
+ if (index.isPrimaryIndex()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes cannot be " + "primary indexes.");
+ } else {
+ compDecompFactory = NoOpCompressorDecompressorFactory.INSTANCE;
+ }
+
+ return new LSMBTreeLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
+ filterCmpFactories, filterFields, opTrackerFactory, ioOpCallbackFactory,
+ pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
+ mergePolicyFactory, mergePolicyProperties, true, null, bloomFilterFalsePositiveRate,
+ index.isPrimaryIndex(), btreeFields, compDecompFactory, false);
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_DATASET_TYPE,
+ dataset.getDatasetType().toString());
+ }
+ }
+
+ private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
+ ARecordType recordType, ARecordType metaType) throws AlgebricksException {
+ ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
+ if (index.isPrimaryIndex()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes cannot be " + "primary indexes.");
+ } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes are not " + "supported for external datasets.");
+ }
+ ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ ITypeTraits[] secondaryTypeTraits;
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys =
+ arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+ int secondaryTypeTraitPos = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ ARecordType sourceType;
+ if (e.getSourceIndicator() == 0) {
+ sourceType = recordType;
+ } else {
+ sourceType = metaType;
+ }
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ List<String> project = e.getProjectList().get(i);
+ Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
+ ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+ IAType keyType = keyTypePair.first;
+ secondaryTypeTraits[secondaryTypeTraitPos++] = typeTraitProvider.getTypeTrait(keyType);
+ }
+ }
+ // Add serializers and comparators for primary index fields.
+ System.arraycopy(primaryTypeTraits, 0, secondaryTypeTraits, numSecondaryKeys, numPrimaryKeys);
+ return secondaryTypeTraits;
+ }
+
+ private static IBinaryComparatorFactory[] getCmpFactories(MetadataProvider metadataProvider, Dataset dataset,
+ Index index, ARecordType recordType, ARecordType metaType) throws AlgebricksException {
+ IBinaryComparatorFactory[] primaryCmpFactories =
+ dataset.getPrimaryComparatorFactories(metadataProvider, recordType, metaType);
+ if (index.isPrimaryIndex()) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes cannot be " + "primary indexes.");
+ } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Array indexes are not " + "supported for external datasets.");
+ }
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys =
+ arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ int numPrimaryKeys = dataset.getPrimaryKeys().size();
+ IBinaryComparatorFactoryProvider cmpFactoryProvider =
+ metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
+ IBinaryComparatorFactory[] secondaryCmpFactories =
+ new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+ int secondaryCmpFactoriesPos = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ ARecordType sourceType;
+ if (e.getSourceIndicator() == 0) {
+ sourceType = recordType;
+ } else {
+ sourceType = metaType;
+ }
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ List<String> project = e.getProjectList().get(i);
+ Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project), sourceType,
+ ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+ IAType keyType = keyTypePair.first;
+ secondaryCmpFactories[secondaryCmpFactoriesPos++] =
+ cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
+ }
+ }
+ // Add serializers and comparators for primary index fields.
+ System.arraycopy(primaryCmpFactories, 0, secondaryCmpFactories, numSecondaryKeys, numPrimaryKeys);
+ return secondaryCmpFactories;
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
index 0ffdedc..7f56f2c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/BTreeResourceFactoryProvider.java
@@ -22,7 +22,6 @@
import java.util.Map;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -129,20 +128,21 @@
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.EXTERNAL_FILE_INDEX_TYPE_TRAITS;
}
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
ITypeTraitProvider typeTraitProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
}
@@ -163,22 +163,23 @@
&& index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.FILES_INDEX_COMP_FACTORIES;
}
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IBinaryComparatorFactoryProvider cmpFactoryProvider =
metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
IBinaryComparatorFactory[] secondaryCmpFactories =
new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
secondaryCmpFactories[i] = cmpFactoryProvider.getBinaryComparatorFactory(keyType, true);
}
@@ -193,23 +194,34 @@
// both the Primary index and the Primary Key index have bloom filters
if (index.isPrimaryIndex() || index.isPrimaryKeyIndex()) {
return dataset.getPrimaryBloomFilterFields();
- } else if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ }
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
if (index.getIndexName().equals(IndexingConstants.getFilesIndexName(dataset.getDatasetName()))) {
return FilesIndexDescription.BLOOM_FILTER_FIELDS;
} else {
- return new int[] { index.getKeyFieldNames().size() };
+ Index.ValueIndexDetails indexDetails = ((Index.ValueIndexDetails) index.getIndexDetails());
+ return new int[] { indexDetails.getKeyFieldNames().size() };
}
- } else if (index.getIndexType() == IndexType.BTREE || index.getIndexType() == IndexType.RTREE) {
- // secondary btrees and rtrees do not have bloom filters
- return null;
- } else {
- // inverted indexes have bloom filters on deleted-key btrees
- int numKeys = index.getKeyFieldNames().size();
- int[] bloomFilterKeyFields = new int[numKeys];
- for (int i = 0; i < numKeys; i++) {
- bloomFilterKeyFields[i] = i;
- }
- return bloomFilterKeyFields;
+ }
+ switch (index.getIndexType()) {
+ case BTREE:
+ case RTREE:
+ // secondary btrees and rtrees do not have bloom filters
+ return null;
+ case LENGTH_PARTITIONED_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case SINGLE_PARTITION_WORD_INVIX:
+ // inverted indexes have bloom filters on deleted-key btrees
+ int numKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
+ int[] bloomFilterKeyFields = new int[numKeys];
+ for (int i = 0; i < numKeys; i++) {
+ bloomFilterKeyFields[i] = i;
+ }
+ return bloomFilterKeyFields;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ String.valueOf(index.getIndexType()));
}
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index 3c02a4f..1376008 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -37,6 +37,8 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.NodeGroup;
import org.apache.asterix.metadata.entities.Synonym;
@@ -47,6 +49,8 @@
import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import com.google.common.base.Strings;
+
public class MetadataManagerUtil {
private MetadataManagerUtil() {
@@ -152,6 +156,21 @@
return MetadataManager.INSTANCE.getSynonym(mdTxnCtx, dataverseName, synonymName);
}
+ public static FullTextConfigMetadataEntity findFullTextConfigDescriptor(MetadataTransactionContext mdTxnCtx,
+ DataverseName dataverseName, String ftConfigName) throws AlgebricksException {
+ // If the config name is null, then the default config will be returned
+ if (Strings.isNullOrEmpty(ftConfigName)) {
+ return FullTextConfigMetadataEntity.getDefaultFullTextConfigMetadataEntity();
+ }
+
+ return MetadataManager.INSTANCE.getFullTextConfig(mdTxnCtx, dataverseName, ftConfigName);
+ }
+
+ public static FullTextFilterMetadataEntity findFullTextFilterDescriptor(MetadataTransactionContext mdTxnCtx,
+ DataverseName dataverseName, String ftFilterName) throws AlgebricksException {
+ return MetadataManager.INSTANCE.getFullTextFilter(mdTxnCtx, dataverseName, ftFilterName);
+ }
+
public static List<Index> getDatasetIndexes(MetadataTransactionContext mdTxnCtx, DataverseName dataverseName,
String datasetName) throws AlgebricksException {
return MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index f18992f..1721975 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -86,12 +86,15 @@
import org.apache.asterix.metadata.entities.Feed;
import org.apache.asterix.metadata.entities.FeedConnection;
import org.apache.asterix.metadata.entities.FeedPolicyEntity;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.Synonym;
import org.apache.asterix.metadata.feeds.FeedMetadataUtil;
import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.metadata.utils.FullTextUtil;
import org.apache.asterix.metadata.utils.MetadataConstants;
import org.apache.asterix.metadata.utils.SplitsAndConstraintsUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -106,7 +109,9 @@
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor;
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
import org.apache.asterix.runtime.operators.LSMPrimaryInsertOperatorDescriptor;
+import org.apache.asterix.runtime.operators.LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor;
import org.apache.asterix.runtime.operators.LSMSecondaryUpsertOperatorDescriptor;
+import org.apache.asterix.runtime.operators.LSMSecondaryUpsertWithNestedPlanOperatorDescriptor;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -131,6 +136,7 @@
import org.apache.hyracks.algebricks.data.IPrinterFactory;
import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -162,6 +168,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeBatchPointSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -450,6 +457,16 @@
return MetadataManagerUtil.findSynonym(mdTxnCtx, dataverseName, synonymName);
}
+ public FullTextConfigMetadataEntity findFullTextConfig(DataverseName dataverseName, String ftConfigName)
+ throws AlgebricksException {
+ return MetadataManagerUtil.findFullTextConfigDescriptor(mdTxnCtx, dataverseName, ftConfigName);
+ }
+
+ public FullTextFilterMetadataEntity findFullTextFilter(DataverseName dataverseName, String ftFilterName)
+ throws AlgebricksException {
+ return MetadataManagerUtil.findFullTextFilterDescriptor(mdTxnCtx, dataverseName, ftFilterName);
+ }
+
@Override
public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
return BuiltinFunctions.getBuiltinFunctionInfo(fid);
@@ -531,6 +548,21 @@
}
Index theIndex = isSecondary ? MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName) : primaryIndex;
+
+ int numSecondaryKeys;
+ switch (theIndex.getIndexType()) {
+ case ARRAY:
+ numSecondaryKeys = ((Index.ArrayIndexDetails) theIndex.getIndexDetails()).getElementList().stream()
+ .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ break;
+ case BTREE:
+ numSecondaryKeys = ((Index.ValueIndexDetails) theIndex.getIndexDetails()).getKeyFieldNames().size();
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ theIndex.getIndexType().toString());
+ }
+
int numPrimaryKeys = dataset.getPrimaryKeys().size();
RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
@@ -545,7 +577,6 @@
byte[] failValueForIndexOnlyPlan = null;
boolean proceedIndexOnlyPlan = isIndexOnlyPlan && isSecondary;
if (proceedIndexOnlyPlan) {
- int numSecondaryKeys = theIndex.getKeyFieldNames().size();
primaryKeyFieldsInSecondaryIndex = new int[numPrimaryKeys];
for (int i = 0; i < numPrimaryKeys; i++) {
primaryKeyFieldsInSecondaryIndex[i] = i + numSecondaryKeys;
@@ -594,6 +625,7 @@
throw new AlgebricksException("Code generation error: no index " + indexName + " for " + dataset() + " "
+ dataset.getDatasetName());
}
+ Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
@@ -607,8 +639,8 @@
byte[] failValueForIndexOnlyPlan = null;
if (isIndexOnlyPlan) {
ARecordType recType = (ARecordType) findType(dataset.getItemTypeDataverseName(), dataset.getItemTypeName());
- List<List<String>> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
- List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+ List<List<String>> secondaryKeyFields = secondaryIndexDetails.getKeyFieldNames();
+ List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
Pair<IAType, Boolean> keyTypePair =
Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
IAType keyType = keyTypePair.first;
@@ -746,10 +778,11 @@
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
- boolean bulkload) throws AlgebricksException {
+ boolean bulkload, List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
+ throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema,
inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
- context, spec, bulkload, null, null, null);
+ context, spec, bulkload, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
}
@Override
@@ -757,11 +790,12 @@
IDataSourceIndex<String, DataSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
- ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+ ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
+ List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema,
inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalNonKeyFields, filterExpr, recordDesc,
- context, spec, false, null, null, null);
+ context, spec, false, null, null, null, secondaryKeysPipelines, pipelineTopSchema);
}
@Override
@@ -771,10 +805,11 @@
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalFilteringKeys,
ILogicalExpression filterExpr, LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
LogicalVariable prevAdditionalFilteringKey, RecordDescriptor recordDesc, JobGenContext context,
- JobSpecification spec) throws AlgebricksException {
+ JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
return getIndexInsertOrDeleteOrUpsertRuntime(IndexOperation.UPSERT, dataSourceIndex, propagatedSchema,
inputSchemas, typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, recordDesc,
- context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey);
+ context, spec, false, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKey,
+ secondaryKeysPipelines, null);
}
@Override
@@ -1179,7 +1214,8 @@
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr,
RecordDescriptor inputRecordDesc, JobGenContext context, JobSpecification spec, boolean bulkload,
LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
- LogicalVariable prevAdditionalFilteringKey) throws AlgebricksException {
+ LogicalVariable prevAdditionalFilteringKey, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
+ IOperatorSchema pipelineTopSchema) throws AlgebricksException {
String indexName = dataSourceIndex.getId();
DataverseName dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
String datasetName = dataSourceIndex.getDataSource().getId().getDatasourceName();
@@ -1193,12 +1229,36 @@
prevAdditionalFilteringKeys = new ArrayList<>();
prevAdditionalFilteringKeys.add(prevAdditionalFilteringKey);
}
- AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+
+ // If we have a pipeline, then we need to pass the schema of the pipeline to the filter factory.
+ AsterixTupleFilterFactory filterFactory;
+ if (pipelineTopSchema != null) {
+ IOperatorSchema[] schemasForFilterFactory = new IOperatorSchema[inputSchemas.length + 1];
+ System.arraycopy(inputSchemas, 0, schemasForFilterFactory, 0, inputSchemas.length);
+ schemasForFilterFactory[inputSchemas.length] = pipelineTopSchema;
+ filterFactory = createTupleFilterFactory(schemasForFilterFactory, typeEnv, filterExpr, context);
+
+ } else {
+ filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+ }
+
switch (secondaryIndex.getIndexType()) {
case BTREE:
return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+ case ARRAY:
+ if (bulkload) {
+ // In the case of bulk-load, we do not handle any nested plans. We perform the exact same behavior
+ // as a normal B-Tree bulk load.
+ return getBTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
+ secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec,
+ indexOp, bulkload, upsertIndicatorVar, prevSecondaryKeys, prevAdditionalFilteringKeys);
+ } else {
+ return getArrayIndexRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
+ additionalNonKeyFields, inputRecordDesc, spec, indexOp, upsertIndicatorVar,
+ secondaryKeysPipelines);
+ }
case RTREE:
return getRTreeRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
secondaryKeys, additionalNonKeyFields, filterFactory, inputRecordDesc, context, spec, indexOp,
@@ -1213,8 +1273,7 @@
prevAdditionalFilteringKeys);
default:
throw new AlgebricksException(
- indexOp.name() + "Insert, upsert, and delete not implemented for index type: "
- + secondaryIndex.getIndexType());
+ indexOp.name() + " not implemented for index type: " + secondaryIndex.getIndexType());
}
}
@@ -1305,6 +1364,60 @@
}
}
+ private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getArrayIndexRuntime(DataverseName dataverseName,
+ String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
+ List<LogicalVariable> additionalNonKeyFields, RecordDescriptor inputRecordDesc, JobSpecification spec,
+ IndexOperation indexOp, LogicalVariable upsertIndicatorVar,
+ List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException {
+
+ Dataset dataset = MetadataManagerUtil.findExistingDataset(mdTxnCtx, dataverseName, datasetName);
+ int numPrimaryKeys = primaryKeys.size();
+ int numFilterFields = DatasetUtil.getFilterField(dataset) == null ? 0 : 1;
+
+ // Generate field permutations (this only includes primary keys and filter fields).
+ int[] fieldPermutation = new int[numPrimaryKeys + numFilterFields];
+ int[] modificationCallbackPrimaryKeyFields = new int[primaryKeys.size()];
+ int i = 0;
+ int j = 0;
+ for (LogicalVariable varKey : primaryKeys) {
+ int idx = propagatedSchema.findVariable(varKey);
+ fieldPermutation[i] = idx;
+ modificationCallbackPrimaryKeyFields[j] = i;
+ i++;
+ j++;
+ }
+ if (numFilterFields > 0) {
+ int idx = propagatedSchema.findVariable(additionalNonKeyFields.get(0));
+ fieldPermutation[numPrimaryKeys] = idx;
+ }
+
+ try {
+ // Index parameters.
+ Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+ dataset.getDatasetName(), indexName);
+ Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
+ getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
+ // Prepare callback.
+ IModificationOperationCallbackFactory modificationCallbackFactory = dataset.getModificationCallbackFactory(
+ storageComponentProvider, secondaryIndex, indexOp, modificationCallbackPrimaryKeyFields);
+ IIndexDataflowHelperFactory idfh = new IndexDataflowHelperFactory(
+ storageComponentProvider.getStorageManager(), splitsAndConstraint.first);
+ IOperatorDescriptor op;
+ if (indexOp == IndexOperation.UPSERT) {
+ int upsertIndicatorFieldIndex = propagatedSchema.findVariable(upsertIndicatorVar);
+ op = new LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(spec, inputRecordDesc, fieldPermutation,
+ idfh, modificationCallbackFactory, upsertIndicatorFieldIndex, BinaryBooleanInspector.FACTORY,
+ secondaryKeysPipelines.get(0), secondaryKeysPipelines.get(1));
+ } else {
+ op = new LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor(spec, inputRecordDesc,
+ fieldPermutation, indexOp, idfh, modificationCallbackFactory, secondaryKeysPipelines.get(0));
+ }
+ return new Pair<>(op, splitsAndConstraint.second);
+ } catch (Exception e) {
+ throw new AlgebricksException(e);
+ }
+ }
+
private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeRuntime(DataverseName dataverseName,
String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
@@ -1320,8 +1433,9 @@
ARecordType recType = (ARecordType) itemType;
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
- List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
- List<IAType> secondaryKeyTypes = secondaryIndex.getKeyFieldTypes();
+ Index.ValueIndexDetails secondaryIndexDetails = (Index.ValueIndexDetails) secondaryIndex.getIndexDetails();
+ List<List<String>> secondaryKeyExprs = secondaryIndexDetails.getKeyFieldNames();
+ List<IAType> secondaryKeyTypes = secondaryIndexDetails.getKeyFieldTypes();
Pair<IAType, Boolean> keyPairType =
Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyExprs.get(0), recType);
IAType spatialType = keyPairType.first;
@@ -1621,9 +1735,10 @@
// Index parameters.
Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), indexName);
+ Index.TextIndexDetails secondaryIndexDetails = (Index.TextIndexDetails) secondaryIndex.getIndexDetails();
- List<List<String>> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
- List<IAType> secondaryKeyTypeEntries = secondaryIndex.getKeyFieldTypes();
+ List<List<String>> secondaryKeyExprs = secondaryIndexDetails.getKeyFieldNames();
+ List<IAType> secondaryKeyTypeEntries = secondaryIndexDetails.getKeyFieldTypes();
int numTokenFields = (!isPartitioned) ? secondaryKeys.size() : secondaryKeys.size() + 1;
ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
@@ -1652,7 +1767,10 @@
}
IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
- secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
+ secondaryKeyType.getTypeTag(), indexType, secondaryIndexDetails.getGramLength());
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+ FullTextUtil.fetchFilterAndCreateConfigEvaluator(this, secondaryIndex.getDataverseName(),
+ secondaryIndexDetails.getFullTextConfigName());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint =
getSplitProviderAndConstraints(dataset, secondaryIndex.getIndexName());
@@ -1694,8 +1812,9 @@
keyFields[k] = k;
}
- tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
- keyFields, isPartitioned, true, false, MissingWriterFactory.INSTANCE);
+ tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, docField, keyFields, isPartitioned, true, false,
+ MissingWriterFactory.INSTANCE);
return new Pair<>(tokenizerOp, splitsAndConstraint.second);
} catch (Exception e) {
throw new AlgebricksException(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index 39a8eff..34e0c41 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -55,6 +55,7 @@
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.metadata.declared.ArrayBTreeResourceFactoryProvider;
import org.apache.asterix.metadata.declared.BTreeResourceFactoryProvider;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.lock.ExternalDatasetsRegistry;
@@ -474,6 +475,11 @@
recordType, metaType, mdProvider.getStorageComponentProvider().getComparatorFactoryProvider());
IResourceFactory resourceFactory;
switch (index.getIndexType()) {
+ case ARRAY:
+ resourceFactory = ArrayBTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
+ recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
+ filterCmpFactories);
+ break;
case BTREE:
resourceFactory = BTreeResourceFactoryProvider.INSTANCE.getResourceFactory(mdProvider, this, index,
recordType, metaType, mergePolicyFactory, mergePolicyProperties, filterTypeTraits,
@@ -577,7 +583,7 @@
return new SecondaryIndexInstanctSearchOperationCallbackFactory(getDatasetId(),
primaryKeyFieldsInSecondaryIndex, storageComponentProvider.getTransactionSubsystemProvider(),
index.resourceType());
- } else if (index.getKeyFieldNames().isEmpty()) {
+ } else if (index.isPrimaryKeyIndex()) {
// this is the case where the index is secondary primary index and locking is required
// since the secondary primary index replaces the dataset index (which locks)
return new PrimaryIndexInstantSearchOperationCallbackFactory(getDatasetId(), primaryKeyFields,
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
new file mode 100644
index 0000000..3ee7106
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextConfigMetadataEntity.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+
+public class FullTextConfigMetadataEntity implements IMetadataEntity<FullTextConfigMetadataEntity> {
+ private final FullTextConfigDescriptor fullTextConfig;
+
+ public FullTextConfigMetadataEntity(FullTextConfigDescriptor config) {
+ this.fullTextConfig = config;
+ }
+
+ public FullTextConfigDescriptor getFullTextConfig() {
+ return fullTextConfig;
+ }
+
+ @Override
+ public FullTextConfigMetadataEntity addToCache(MetadataCache cache) {
+ return cache.addFullTextConfigIfNotExists(this);
+ }
+
+ @Override
+ public FullTextConfigMetadataEntity dropFromCache(MetadataCache cache) {
+ return cache.dropFullTextConfig(this);
+ }
+
+ public static FullTextConfigMetadataEntity getDefaultFullTextConfigMetadataEntity() {
+ return new FullTextConfigMetadataEntity(FullTextConfigDescriptor.getDefaultFullTextConfig());
+ }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
new file mode 100644
index 0000000..eb4c7bc
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/FullTextFilterMetadataEntity.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.metadata.entities;
+
+import org.apache.asterix.metadata.MetadataCache;
+import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+
+public class FullTextFilterMetadataEntity implements IMetadataEntity<FullTextFilterMetadataEntity> {
+ private final AbstractFullTextFilterDescriptor filterDescriptor;
+
+ public FullTextFilterMetadataEntity(AbstractFullTextFilterDescriptor filterDescriptor) {
+ this.filterDescriptor = filterDescriptor;
+ }
+
+ @Override
+ public FullTextFilterMetadataEntity addToCache(MetadataCache cache) {
+ return cache.addFullTextFilterIfNotExists(this);
+ }
+
+ @Override
+ public FullTextFilterMetadataEntity dropFromCache(MetadataCache cache) {
+ return cache.dropFullTextFilter(this);
+ }
+
+ public AbstractFullTextFilterDescriptor getFullTextFilter() {
+ return filterDescriptor;
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
index ee8622c..cf09779 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Index.java
@@ -19,7 +19,10 @@
package org.apache.asterix.metadata.entities;
+import java.io.Serializable;
+import java.util.Collections;
import java.util.List;
+import java.util.Objects;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.AsterixException;
@@ -41,8 +44,9 @@
*/
public class Index implements IMetadataEntity<Index>, Comparable<Index> {
- private static final long serialVersionUID = 2L;
+ private static final long serialVersionUID = 3L;
public static final int RECORD_INDICATOR = 0;
+ public static final int META_RECORD_INDICATOR = 1;
private final DataverseName dataverseName;
// Enforced to be unique within a dataverse.
@@ -50,39 +54,38 @@
// Enforced to be unique within a dataverse, dataset combination.
private final String indexName;
private final IndexType indexType;
- private final List<List<String>> keyFieldNames;
- private final List<Integer> keyFieldSourceIndicators;
- private final List<IAType> keyFieldTypes;
- private final boolean overrideKeyFieldTypes;
- private final boolean isEnforced;
+ private final IIndexDetails indexDetails;
private final boolean isPrimaryIndex;
- // Specific to NGRAM indexes.
- private final int gramLength;
+ private final boolean isEnforced;
// Type of pending operations with respect to atomic DDL operation
private int pendingOp;
public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
- List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
- int gramLength, boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
- this.dataverseName = dataverseName;
- this.datasetName = datasetName;
- this.indexName = indexName;
+ IIndexDetails indexDetails, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
+ boolean categoryOk = (indexType == null && indexDetails == null) || (IndexCategory
+ .of(Objects.requireNonNull(indexType)) == ((AbstractIndexDetails) Objects.requireNonNull(indexDetails))
+ .getIndexCategory());
+ if (!categoryOk) {
+ throw new IllegalArgumentException();
+ }
+ this.dataverseName = Objects.requireNonNull(dataverseName);
+ this.datasetName = Objects.requireNonNull(datasetName);
+ this.indexName = Objects.requireNonNull(indexName);
this.indexType = indexType;
- this.keyFieldNames = keyFieldNames;
- this.keyFieldSourceIndicators = keyFieldSourceIndicators;
- this.keyFieldTypes = keyFieldTypes;
- this.gramLength = gramLength;
- this.overrideKeyFieldTypes = overrideKeyFieldTypes;
- this.isEnforced = isEnforced;
+ this.indexDetails = indexDetails;
this.isPrimaryIndex = isPrimaryIndex;
+ this.isEnforced = isEnforced;
this.pendingOp = pendingOp;
}
+ @Deprecated
public Index(DataverseName dataverseName, String datasetName, String indexName, IndexType indexType,
List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes,
boolean overrideKeyFieldTypes, boolean isEnforced, boolean isPrimaryIndex, int pendingOp) {
- this(dataverseName, datasetName, indexName, indexType, keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
- -1, overrideKeyFieldTypes, isEnforced, isPrimaryIndex, pendingOp);
+ this(dataverseName,
+ datasetName, indexName, indexType, createSimpleIndexDetails(indexType, keyFieldNames,
+ keyFieldSourceIndicators, keyFieldTypes, overrideKeyFieldTypes),
+ isEnforced, isPrimaryIndex, pendingOp);
}
public DataverseName getDataverseName() {
@@ -97,22 +100,6 @@
return indexName;
}
- public List<List<String>> getKeyFieldNames() {
- return keyFieldNames;
- }
-
- public List<Integer> getKeyFieldSourceIndicators() {
- return keyFieldSourceIndicators;
- }
-
- public List<IAType> getKeyFieldTypes() {
- return keyFieldTypes;
- }
-
- public int getGramLength() {
- return gramLength;
- }
-
public IndexType getIndexType() {
return indexType;
}
@@ -121,8 +108,8 @@
return isPrimaryIndex;
}
- public boolean isOverridingKeyFieldTypes() {
- return overrideKeyFieldTypes;
+ public IIndexDetails getIndexDetails() {
+ return indexDetails;
}
public boolean isEnforced() {
@@ -143,7 +130,7 @@
public boolean isPrimaryKeyIndex() {
// a primary key index has no key field names
- return keyFieldNames.isEmpty();
+ return indexType == IndexType.BTREE && ((ValueIndexDetails) indexDetails).keyFieldNames.isEmpty();
}
public static Pair<IAType, Boolean> getNonNullableType(IAType keyType) {
@@ -193,9 +180,7 @@
}
private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
- IAType fieldType = recType;
- fieldType = recType.getSubFieldType(expr);
- return fieldType;
+ return recType.getSubFieldType(expr);
}
@Override
@@ -269,19 +254,9 @@
return dataverseName.compareTo(otherIndex.getDataverseName());
}
- public boolean hasMetaFields() {
- if (keyFieldSourceIndicators != null) {
- for (Integer indicator : keyFieldSourceIndicators) {
- if (indicator.intValue() != 0) {
- return true;
- }
- }
- }
- return false;
- }
-
public byte resourceType() throws CompilationException {
switch (indexType) {
+ case ARRAY:
case BTREE:
return ResourceType.LSM_BTREE;
case RTREE:
@@ -300,4 +275,225 @@
public String toString() {
return dataverseName + "." + datasetName + "." + indexName;
}
+
+ public enum IndexCategory {
+ VALUE,
+ TEXT,
+ ARRAY;
+
+ public static IndexCategory of(IndexType indexType) {
+ switch (indexType) {
+ case BTREE:
+ case RTREE:
+ return VALUE;
+ case SINGLE_PARTITION_WORD_INVIX:
+ case SINGLE_PARTITION_NGRAM_INVIX:
+ case LENGTH_PARTITIONED_WORD_INVIX:
+ case LENGTH_PARTITIONED_NGRAM_INVIX:
+ return TEXT;
+ case ARRAY:
+ return ARRAY;
+ default:
+ throw new IllegalArgumentException(String.valueOf(indexType));
+ }
+ }
+ }
+
+ public interface IIndexDetails extends Serializable {
+ boolean isOverridingKeyFieldTypes();
+ }
+
+ static abstract class AbstractIndexDetails implements IIndexDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ abstract IndexCategory getIndexCategory();
+ }
+
+ public static final class ValueIndexDetails extends AbstractIndexDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<List<String>> keyFieldNames;
+
+ private final List<Integer> keyFieldSourceIndicators;
+
+ private final List<IAType> keyFieldTypes;
+
+ private final boolean overrideKeyFieldTypes;
+
+ public ValueIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+ List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+ this.keyFieldNames = keyFieldNames;
+ this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+ this.keyFieldTypes = keyFieldTypes;
+ this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+ }
+
+ @Override
+ IndexCategory getIndexCategory() {
+ return IndexCategory.VALUE;
+ }
+
+ public List<List<String>> getKeyFieldNames() {
+ return keyFieldNames;
+ }
+
+ public List<Integer> getKeyFieldSourceIndicators() {
+ return keyFieldSourceIndicators;
+ }
+
+ public List<IAType> getKeyFieldTypes() {
+ return keyFieldTypes;
+ }
+
+ @Override
+ public boolean isOverridingKeyFieldTypes() {
+ return overrideKeyFieldTypes;
+ }
+ }
+
+ public static final class TextIndexDetails extends AbstractIndexDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<List<String>> keyFieldNames;
+
+ private final List<Integer> keyFieldSourceIndicators;
+
+ private final List<IAType> keyFieldTypes;
+
+ private final boolean overrideKeyFieldTypes;
+
+ // ToDo: to allow index to access the full-text config in another dataverse,
+ // maybe we need to add a new field here fullTextConfigDataverseName for dataverse name of full-text config
+ // Specific to FullText indexes.
+ private final String fullTextConfigName;
+
+ // Specific to NGRAM indexes.
+ private final int gramLength;
+
+ public TextIndexDetails(List<List<String>> keyFieldNames, List<Integer> keyFieldSourceIndicators,
+ List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes, int gramLength, String fullTextConfigName) {
+ this.keyFieldNames = keyFieldNames;
+ this.keyFieldTypes = keyFieldTypes;
+ this.keyFieldSourceIndicators = keyFieldSourceIndicators;
+ this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+ this.gramLength = gramLength;
+ this.fullTextConfigName = fullTextConfigName;
+ }
+
+ @Override
+ IndexCategory getIndexCategory() {
+ return IndexCategory.TEXT;
+ }
+
+ public List<List<String>> getKeyFieldNames() {
+ return keyFieldNames;
+ }
+
+ public List<Integer> getKeyFieldSourceIndicators() {
+ return keyFieldSourceIndicators;
+ }
+
+ public List<IAType> getKeyFieldTypes() {
+ return keyFieldTypes;
+ }
+
+ @Override
+ public boolean isOverridingKeyFieldTypes() {
+ return overrideKeyFieldTypes;
+ }
+
+ public int getGramLength() {
+ return gramLength;
+ }
+
+ public String getFullTextConfigName() {
+ return fullTextConfigName;
+ }
+ }
+
+ public static class ArrayIndexDetails extends AbstractIndexDetails {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<ArrayIndexElement> elementList;
+
+ private final boolean overrideKeyFieldTypes;
+
+ public ArrayIndexDetails(List<ArrayIndexElement> elementList, boolean overrideKeyFieldTypes) {
+ this.elementList = elementList;
+ this.overrideKeyFieldTypes = overrideKeyFieldTypes;
+ }
+
+ @Override
+ IndexCategory getIndexCategory() {
+ return IndexCategory.ARRAY;
+ }
+
+ public List<ArrayIndexElement> getElementList() {
+ return elementList;
+ }
+
+ @Override
+ public boolean isOverridingKeyFieldTypes() {
+ return overrideKeyFieldTypes;
+ }
+ }
+
+ public static final class ArrayIndexElement implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final List<List<String>> unnestList;
+
+ private final List<List<String>> projectList;
+
+ private final List<IAType> typeList;
+
+ private final int sourceIndicator;
+
+ public ArrayIndexElement(List<List<String>> unnestList, List<List<String>> projectList, List<IAType> typeList,
+ int sourceIndicator) {
+ this.unnestList = unnestList != null ? unnestList : Collections.emptyList();
+ this.projectList = projectList;
+ this.typeList = typeList;
+ this.sourceIndicator = sourceIndicator;
+ }
+
+ public List<List<String>> getUnnestList() {
+ return unnestList;
+ }
+
+ public List<List<String>> getProjectList() {
+ return projectList;
+ }
+
+ public List<IAType> getTypeList() {
+ return typeList;
+ }
+
+ public int getSourceIndicator() {
+ return sourceIndicator;
+ }
+ }
+
+ @Deprecated
+ private static Index.IIndexDetails createSimpleIndexDetails(IndexType indexType, List<List<String>> keyFieldNames,
+ List<Integer> keyFieldSourceIndicators, List<IAType> keyFieldTypes, boolean overrideKeyFieldTypes) {
+ if (indexType == null) {
+ return null;
+ }
+ switch (Index.IndexCategory.of(indexType)) {
+ case VALUE:
+ return new ValueIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
+ overrideKeyFieldTypes);
+ case TEXT:
+ return new TextIndexDetails(keyFieldNames, keyFieldSourceIndicators, keyFieldTypes,
+ overrideKeyFieldTypes, -1, null);
+ default:
+ throw new IllegalArgumentException(String.valueOf(indexType));
+ }
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java
new file mode 100644
index 0000000..ec5cdc9
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextConfigMetadataEntityTupleTranslator.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX;
+
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.FullTextConfigMetadataEntity;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.commons.lang3.EnumUtils;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextConfigMetadataEntityTupleTranslator extends AbstractTupleTranslator<FullTextConfigMetadataEntity> {
+
+ private static final int FULL_TEXT_CONFIG_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+ protected final ArrayTupleReference tuple;
+ protected final ISerializerDeserializer<AInt8> int8Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+
+ protected FullTextConfigMetadataEntityTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.FULL_TEXT_CONFIG_DATASET, FULL_TEXT_CONFIG_PAYLOAD_TUPLE_FIELD_INDEX);
+ if (getTuple) {
+ tuple = new ArrayTupleReference();
+ } else {
+ tuple = null;
+ }
+ }
+
+ @Override
+ protected FullTextConfigMetadataEntity createMetadataEntityFromARecord(ARecord aRecord)
+ throws HyracksDataException, AlgebricksException {
+ DataverseName dataverseName = DataverseName.createFromCanonicalForm(
+ ((AString) aRecord.getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX))
+ .getStringValue());
+
+ String name = ((AString) aRecord.getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX))
+ .getStringValue();
+
+ TokenizerCategory tokenizerCategory =
+ EnumUtils.getEnumIgnoreCase(TokenizerCategory.class,
+ ((AString) aRecord
+ .getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX))
+ .getStringValue());
+
+ ImmutableList.Builder<String> filterNamesBuilder = ImmutableList.builder();
+ IACursor filterNamesCursor = ((AOrderedList) (aRecord
+ .getValueByPos(MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX))).getCursor();
+ while (filterNamesCursor.next()) {
+ filterNamesBuilder.add(((AString) filterNamesCursor.get()).getStringValue());
+ }
+
+ FullTextConfigDescriptor configDescriptor =
+ new FullTextConfigDescriptor(dataverseName, name, tokenizerCategory, filterNamesBuilder.build());
+ FullTextConfigMetadataEntity configMetadataEntity = new FullTextConfigMetadataEntity(configDescriptor);
+ return configMetadataEntity;
+ }
+
+ private void writeIndex(String dataverseName, String configName, ArrayTupleBuilder tupleBuilder)
+ throws HyracksDataException {
+ aString.setValue(dataverseName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(configName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(FullTextConfigMetadataEntity configMetadataEntity)
+ throws HyracksDataException {
+ tupleBuilder.reset();
+
+ FullTextConfigDescriptor configDescriptor = configMetadataEntity.getFullTextConfig();
+
+ writeIndex(configDescriptor.getDataverseName().getCanonicalForm(), configDescriptor.getName(), tupleBuilder);
+
+ recordBuilder.reset(MetadataRecordTypes.FULL_TEXT_CONFIG_RECORDTYPE);
+
+ // write dataverse name
+ fieldValue.reset();
+ aString.setValue(configDescriptor.getDataverseName().getCanonicalForm());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+ // write name
+ fieldValue.reset();
+ aString.setValue(configDescriptor.getName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_CONFIG_NAME_FIELD_INDEX, fieldValue);
+
+ // write tokenizer category
+ fieldValue.reset();
+ aString.setValue(configDescriptor.getTokenizerCategory().name());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_CONFIG_TOKENIZER_FIELD_INDEX, fieldValue);
+
+ // set filter pipeline
+ List<String> filterNames = configDescriptor.getFilterNames();
+
+ OrderedListBuilder listBuilder = new OrderedListBuilder();
+ listBuilder.reset(new AOrderedListType(BuiltinType.ASTRING, null));
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+ for (String s : filterNames) {
+ itemValue.reset();
+ aString.setValue(s);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_PIPELINE_FIELD_INDEX, fieldValue);
+
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java
new file mode 100644
index 0000000..2212e82
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/FullTextFilterMetadataEntityTupleTranslator.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.metadata.entitytupletranslators;
+
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FIELD_NAME_FULL_TEXT_STOPWORD_LIST;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX;
+import static org.apache.asterix.metadata.bootstrap.MetadataRecordTypes.FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX;
+
+import java.util.List;
+
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
+import org.apache.asterix.metadata.entities.FullTextFilterMetadataEntity;
+import org.apache.asterix.om.base.AInt8;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.StopwordsFullTextFilterDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextFilterMetadataEntityTupleTranslator extends AbstractTupleTranslator<FullTextFilterMetadataEntity> {
+
+ private static final int FULLTEXT_FILTER_PAYLOAD_TUPLE_FIELD_INDEX = 2;
+ protected final ArrayTupleReference tuple;
+ protected final ISerializerDeserializer<AInt8> int8Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+
+ protected FullTextFilterMetadataEntityTupleTranslator(boolean getTuple) {
+ super(getTuple, MetadataPrimaryIndexes.FULL_TEXT_FILTER_DATASET, FULLTEXT_FILTER_PAYLOAD_TUPLE_FIELD_INDEX);
+ if (getTuple) {
+ tuple = new ArrayTupleReference();
+ } else {
+ tuple = null;
+ }
+ }
+
+ @Override
+ protected FullTextFilterMetadataEntity createMetadataEntityFromARecord(ARecord aRecord) throws AlgebricksException {
+ AString dataverseName = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX);
+ AString filterName = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX);
+ AString filterTypeAString = (AString) aRecord.getValueByPos(FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX);
+
+ FullTextFilterType filterType = FullTextFilterType.getEnumIgnoreCase(filterTypeAString.getStringValue());
+ AbstractFullTextFilterDescriptor filterDescriptor;
+ switch (filterType) {
+ case STOPWORDS:
+ return createStopwordsFilterDescriptorFromARecord(dataverseName, filterName, aRecord);
+ case STEMMER:
+ case SYNONYM:
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, "Not supported yet");
+ }
+ }
+
+ public FullTextFilterMetadataEntity createStopwordsFilterDescriptorFromARecord(AString dataverseName, AString name,
+ ARecord aRecord) throws AlgebricksException {
+ ImmutableList.Builder<String> stopwordsBuilder = ImmutableList.<String> builder();
+ IACursor stopwordsCursor = ((AOrderedList) (aRecord
+ .getValueByPos(MetadataRecordTypes.FULLTEXT_ENTITY_ARECORD_STOPWORD_LIST_FIELD_INDEX))).getCursor();
+ while (stopwordsCursor.next()) {
+ stopwordsBuilder.add(((AString) stopwordsCursor.get()).getStringValue());
+ }
+
+ StopwordsFullTextFilterDescriptor filterDescriptor = new StopwordsFullTextFilterDescriptor(
+ DataverseName.createFromCanonicalForm(dataverseName.getStringValue()), name.getStringValue(),
+ stopwordsBuilder.build());
+ return new FullTextFilterMetadataEntity(filterDescriptor);
+ }
+
+ private void writeKeyAndValue2FieldVariables(String key, String value) throws HyracksDataException {
+ fieldName.reset();
+ aString.setValue(key);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+
+ fieldValue.reset();
+ aString.setValue(value);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ }
+
+ private void writeOrderedList2RecordBuilder(String strFieldName, List<String> list) throws HyracksDataException {
+ fieldName.reset();
+ aString.setValue(strFieldName);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+
+ OrderedListBuilder listBuilder = new OrderedListBuilder();
+ listBuilder.reset(new AOrderedListType(BuiltinType.ASTRING, null));
+ ArrayBackedValueStorage itemValue = new ArrayBackedValueStorage();
+ for (String s : list) {
+ itemValue.reset();
+ aString.setValue(s);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+
+ recordBuilder.addField(fieldName, fieldValue);
+ }
+
+ private void writeStopwordFilterDescriptor(StopwordsFullTextFilterDescriptor stopwordsFullTextFilterDescriptor)
+ throws HyracksDataException {
+ writeOrderedList2RecordBuilder(FIELD_NAME_FULL_TEXT_STOPWORD_LIST,
+ stopwordsFullTextFilterDescriptor.getStopwordList());
+ }
+
+ private void writeFulltextFilter(AbstractFullTextFilterDescriptor filterDescriptor)
+ throws AsterixException, HyracksDataException {
+ fieldValue.reset();
+ aString.setValue(filterDescriptor.getDataverseName().getCanonicalForm());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_DATAVERSE_NAME_FIELD_INDEX, fieldValue);
+
+ fieldValue.reset();
+ aString.setValue(filterDescriptor.getName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_NAME_FIELD_INDEX, fieldValue);
+
+ fieldValue.reset();
+ aString.setValue(filterDescriptor.getFilterType().getValue());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(FULL_TEXT_ARECORD_FILTER_TYPE_FIELD_INDEX, fieldValue);
+
+ switch (filterDescriptor.getFilterType()) {
+ case STOPWORDS:
+ writeStopwordFilterDescriptor((StopwordsFullTextFilterDescriptor) filterDescriptor);
+ break;
+ case STEMMER:
+ case SYNONYM:
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, "Not supported yet");
+ }
+ }
+
+ private void writeIndex(String dataverseName, String filterName, ArrayTupleBuilder tupleBuilder)
+ throws HyracksDataException {
+ aString.setValue(dataverseName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+
+ aString.setValue(filterName);
+ stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+ tupleBuilder.addFieldEndOffset();
+ }
+
+ @Override
+ public ITupleReference getTupleFromMetadataEntity(FullTextFilterMetadataEntity filterMetadataEntity)
+ throws HyracksDataException, AsterixException {
+ tupleBuilder.reset();
+
+ writeIndex(filterMetadataEntity.getFullTextFilter().getDataverseName().getCanonicalForm(),
+ filterMetadataEntity.getFullTextFilter().getName(), tupleBuilder);
+
+ // Write the record
+ recordBuilder.reset(MetadataRecordTypes.FULL_TEXT_FILTER_RECORDTYPE);
+
+ writeFulltextFilter(filterMetadataEntity.getFullTextFilter());
+
+ recordBuilder.write(tupleBuilder.getDataOutput(), true);
+ tupleBuilder.addFieldEndOffset();
+
+ tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ return tuple;
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 79ce2a4..32764b2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -21,10 +21,16 @@
import java.util.ArrayList;
import java.util.Calendar;
+import java.util.Collections;
import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -40,20 +46,28 @@
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt8;
import org.apache.asterix.om.base.AMutableInt8;
+import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import com.google.common.base.Strings;
+
/**
* Translates an Index metadata entity to an ITupleReference and vice versa.
*/
@@ -64,21 +78,29 @@
// Field name of open field.
public static final String GRAM_LENGTH_FIELD_NAME = "GramLength";
+ public static final String FULL_TEXT_CONFIG_FIELD_NAME = "FullTextConfig";
public static final String INDEX_SEARCHKEY_TYPE_FIELD_NAME = "SearchKeyType";
public static final String INDEX_ISENFORCED_FIELD_NAME = "IsEnforced";
public static final String INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME = "SearchKeySourceIndicator";
+ public static final String INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME = "SearchKeyElements";
+ public static final String COMPLEXSEARCHKEY_UNNEST_FIELD_NAME = "UnnestList";
+ public static final String COMPLEXSEARCHKEY_PROJECT_FIELD_NAME = "ProjectList";
protected final TxnId txnId;
protected final MetadataNode metadataNode;
protected OrderedListBuilder listBuilder;
+ protected OrderedListBuilder innerListBuilder;
protected OrderedListBuilder primaryKeyListBuilder;
+ protected OrderedListBuilder complexSearchKeyNameListBuilder;
+ protected IARecordBuilder complexSearchKeyNameRecordBuilder;
protected AOrderedListType stringList;
protected AOrderedListType int8List;
protected ArrayBackedValueStorage nameValue;
protected ArrayBackedValueStorage itemValue;
protected AMutableInt8 aInt8;
protected ISerializerDeserializer<AInt8> int8Serde;
+ protected ISerializerDeserializer<ANull> nullSerde;
@SuppressWarnings("unchecked")
protected IndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
@@ -87,13 +109,17 @@
this.metadataNode = metadataNode;
if (getTuple) {
listBuilder = new OrderedListBuilder();
+ innerListBuilder = new OrderedListBuilder();
primaryKeyListBuilder = new OrderedListBuilder();
+ complexSearchKeyNameRecordBuilder = new RecordBuilder();
+ complexSearchKeyNameListBuilder = new OrderedListBuilder();
stringList = new AOrderedListType(BuiltinType.ASTRING, null);
int8List = new AOrderedListType(BuiltinType.AINT8, null);
nameValue = new ArrayBackedValueStorage();
itemValue = new ArrayBackedValueStorage();
aInt8 = new AMutableInt8((byte) 0);
int8Serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+ nullSerde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
}
}
@@ -109,55 +135,114 @@
String indexName =
((AString) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXNAME_FIELD_INDEX))
.getStringValue();
- IndexType indexStructure = IndexType.valueOf(
+ IndexType indexType = IndexType.valueOf(
((AString) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX))
.getStringValue());
- IACursor fieldNameCursor =
- ((AOrderedList) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX))
- .getCursor();
- List<List<String>> searchKey = new ArrayList<>();
- AOrderedList fieldNameList;
- while (fieldNameCursor.next()) {
- fieldNameList = (AOrderedList) fieldNameCursor.get();
- IACursor nestedFieldNameCursor = (fieldNameList.getCursor());
- List<String> nestedFieldName = new ArrayList<>();
- while (nestedFieldNameCursor.next()) {
- nestedFieldName.add(((AString) nestedFieldNameCursor.get()).getStringValue());
- }
- searchKey.add(nestedFieldName);
- }
- int indexKeyTypeFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
- IACursor fieldTypeCursor = new ACollectionCursor();
- if (indexKeyTypeFieldPos > 0) {
- fieldTypeCursor = ((AOrderedList) indexRecord.getValueByPos(indexKeyTypeFieldPos)).getCursor();
- }
- List<IAType> searchKeyType = new ArrayList<>(searchKey.size());
- while (fieldTypeCursor.next()) {
- String typeName = ((AString) fieldTypeCursor.get()).getStringValue();
- IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName);
- searchKeyType.add(fieldType);
- }
- boolean isOverridingKeyTypes = !searchKeyType.isEmpty();
- int isEnforcedFieldPos = indexRecord.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
- Boolean isEnforcingKeys = false;
- if (isEnforcedFieldPos > 0) {
- isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
+ // Read key names
+ List<Pair<List<List<String>>, List<List<String>>>> searchElements = new ArrayList<>();
+ switch (Index.IndexCategory.of(indexType)) {
+ case VALUE:
+ case TEXT:
+ // Read the key names from the SearchKeyName field
+ IACursor fieldNameCursor = ((AOrderedList) indexRecord
+ .getValueByPos(MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX)).getCursor();
+ AOrderedList fieldNameList;
+ while (fieldNameCursor.next()) {
+ fieldNameList = (AOrderedList) fieldNameCursor.get();
+ IACursor nestedFieldNameCursor = (fieldNameList.getCursor());
+ List<String> nestedFieldName = new ArrayList<>();
+ while (nestedFieldNameCursor.next()) {
+ nestedFieldName.add(((AString) nestedFieldNameCursor.get()).getStringValue());
+ }
+ searchElements.add(new Pair<>(null, Collections.singletonList(nestedFieldName)));
+ }
+ break;
+ case ARRAY:
+ // Read the unnest/project from the ComplexSearchKeyName field
+ int complexSearchKeyFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME);
+ IACursor complexSearchKeyCursor = new ACollectionCursor();
+ if (complexSearchKeyFieldPos > 0) {
+ complexSearchKeyCursor =
+ ((AOrderedList) indexRecord.getValueByPos(complexSearchKeyFieldPos)).getCursor();
+ }
+ while (complexSearchKeyCursor.next()) {
+ Pair<List<List<String>>, List<List<String>>> searchElement;
+ IAObject complexSearchKeyItem = complexSearchKeyCursor.get();
+ switch (complexSearchKeyItem.getType().getTypeTag()) {
+ case ARRAY:
+ AOrderedList complexSearchKeyArray = (AOrderedList) complexSearchKeyItem;
+ List<String> project = new ArrayList<>(complexSearchKeyArray.size());
+ // We only have one element.
+ AOrderedList innerListForArray = (AOrderedList) complexSearchKeyArray.getItem(0);
+ IACursor innerListCursorForArray = innerListForArray.getCursor();
+ while (innerListCursorForArray.next()) {
+ project.add(((AString) innerListCursorForArray.get()).getStringValue());
+ }
+ searchElement = new Pair<>(null, Collections.singletonList(project));
+ break;
+ case OBJECT:
+ ARecord complexSearchKeyRecord = (ARecord) complexSearchKeyItem;
+ ARecordType complexSearchKeyRecordType = complexSearchKeyRecord.getType();
+ int unnestFieldPos =
+ complexSearchKeyRecordType.getFieldIndex(COMPLEXSEARCHKEY_UNNEST_FIELD_NAME);
+ if (unnestFieldPos < 0) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, complexSearchKeyRecord.toJSON());
+ }
+ AOrderedList unnestFieldList =
+ (AOrderedList) complexSearchKeyRecord.getValueByPos(unnestFieldPos);
+ List<List<String>> unnestList = new ArrayList<>(unnestFieldList.size());
+ IACursor unnestFieldListCursor = unnestFieldList.getCursor();
+ while (unnestFieldListCursor.next()) {
+ AOrderedList innerList = (AOrderedList) unnestFieldListCursor.get();
+ List<String> unnestPath = new ArrayList<>(innerList.size());
+ IACursor innerListCursor = innerList.getCursor();
+ while (innerListCursor.next()) {
+ unnestPath.add(((AString) innerListCursor.get()).getStringValue());
+ }
+ unnestList.add(unnestPath);
+ }
+ int projectFieldPos =
+ complexSearchKeyRecordType.getFieldIndex(COMPLEXSEARCHKEY_PROJECT_FIELD_NAME);
+ List<List<String>> projectList = new ArrayList<>();
+ if (projectFieldPos >= 0) {
+ AOrderedList projectFieldList =
+ (AOrderedList) complexSearchKeyRecord.getValueByPos(projectFieldPos);
+ projectList = new ArrayList<>(projectFieldList.size());
+ IACursor projectFieldListCursor = projectFieldList.getCursor();
+ while (projectFieldListCursor.next()) {
+ AOrderedList innerList = (AOrderedList) projectFieldListCursor.get();
+ List<String> projectPath = new ArrayList<>(innerList.size());
+ IACursor innerListCursor = innerList.getCursor();
+ while (innerListCursor.next()) {
+ projectPath.add(((AString) innerListCursor.get()).getStringValue());
+ }
+ projectList.add(projectPath);
+ }
+ } else {
+ projectList.add(null);
+ }
+ searchElement = new Pair<>(unnestList, projectList);
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, complexSearchKeyItem.toJSON());
+ }
+ searchElements.add(searchElement);
+ }
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
}
- Boolean isPrimaryIndex =
- ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
- .getBoolean();
- int pendingOp = ((AInt32) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
- .getIntegerValue();
- // Check if there is a gram length as well.
- int gramLength = -1;
- int gramLenPos = indexRecord.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
- if (gramLenPos >= 0) {
- gramLength = ((AInt32) indexRecord.getValueByPos(gramLenPos)).getIntegerValue();
+ int searchElementCount = searchElements.size();
+
+ String fullTextConfig = null;
+ int fullTextConfigPos = indexRecord.getType().getFieldIndex(FULL_TEXT_CONFIG_FIELD_NAME);
+ if (fullTextConfigPos >= 0) {
+ fullTextConfig = ((AString) indexRecord.getValueByPos(fullTextConfigPos)).getStringValue();
}
// Read a field-source-indicator field.
- List<Integer> keyFieldSourceIndicator = new ArrayList<>();
+ List<Integer> keyFieldSourceIndicator = new ArrayList<>(searchElementCount);
int keyFieldSourceIndicatorIndex =
indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_SOURCE_INDICATOR_FIELD_NAME);
if (keyFieldSourceIndicatorIndex >= 0) {
@@ -166,14 +251,47 @@
keyFieldSourceIndicator.add((int) ((AInt8) cursor.get()).getByteValue());
}
} else {
- for (int index = 0; index < searchKey.size(); ++index) {
- keyFieldSourceIndicator.add(0);
+ for (int index = 0; index < searchElementCount; ++index) {
+ keyFieldSourceIndicator.add(Index.RECORD_INDICATOR);
}
}
- // index key type information is not persisted, thus we extract type information
- // from the record metadata
+ // Read key types
+ int indexKeyTypeFieldPos = indexRecord.getType().getFieldIndex(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+ IACursor fieldTypeCursor = new ACollectionCursor();
+ if (indexKeyTypeFieldPos > 0) {
+ fieldTypeCursor = ((AOrderedList) indexRecord.getValueByPos(indexKeyTypeFieldPos)).getCursor();
+ }
+ List<List<IAType>> searchKeyType = new ArrayList<>(searchElementCount);
+ while (fieldTypeCursor.next()) {
+ IAObject fieldTypeItem = fieldTypeCursor.get();
+ switch (fieldTypeItem.getType().getTypeTag()) {
+ case STRING:
+ // This is a simple element, place in a single-element list.
+ String typeName = ((AString) fieldTypeItem).getStringValue();
+ IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName);
+ searchKeyType.add(Collections.singletonList(fieldType));
+ break;
+ case ARRAY:
+ // This is a complex element, read all types.
+ List<IAType> fieldTypes = new ArrayList<>();
+ AOrderedList fieldTypeList = (AOrderedList) fieldTypeItem;
+ IACursor fieldTypeListCursor = fieldTypeList.getCursor();
+ while (fieldTypeListCursor.next()) {
+ typeName = ((AString) fieldTypeListCursor.get()).getStringValue();
+ fieldTypes
+ .add(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, typeName));
+ }
+ searchKeyType.add(fieldTypes);
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, fieldTypeItem.toJSON());
+ }
+ }
+ boolean isOverridingKeyTypes;
if (searchKeyType.isEmpty()) {
+ // if index key type information is not persisted, then we extract type information
+ // from the record metadata
Dataset dataset = metadataNode.getDataset(txnId, dataverseName, datasetName);
String datatypeName = dataset.getItemTypeName();
DataverseName datatypeDataverseName = dataset.getItemTypeDataverseName();
@@ -186,15 +304,127 @@
metaDt = (ARecordType) metadataNode.getDatatype(txnId, metatypeDataverseName, metatypeName)
.getDatatype();
}
- searchKeyType = KeyFieldTypeUtil.getKeyTypes(recordDt, metaDt, searchKey, keyFieldSourceIndicator);
+ searchKeyType = new ArrayList<>(searchElementCount);
+ for (int i = 0; i < searchElementCount; i++) {
+ Pair<List<List<String>>, List<List<String>>> searchElement = searchElements.get(i);
+ List<List<String>> unnestPathList = searchElement.first;
+ List<List<String>> projectPathList = searchElement.second;
+
+ ARecordType sourceRecordType = keyFieldSourceIndicator.get(i) == 1 ? metaDt : recordDt;
+ IAType inputTypePrime;
+ boolean inputTypeNullable, inputTypeMissable;
+ if (unnestPathList == null) {
+ inputTypePrime = sourceRecordType;
+ inputTypeNullable = inputTypeMissable = false;
+ } else {
+ Triple<IAType, Boolean, Boolean> unnestTypeResult =
+ KeyFieldTypeUtil.getKeyUnnestType(sourceRecordType, unnestPathList, null);
+ if (unnestTypeResult == null) {
+ inputTypePrime = null; // = ANY
+ inputTypeNullable = inputTypeMissable = true;
+ } else {
+ inputTypePrime = unnestTypeResult.first;
+ inputTypeNullable = unnestTypeResult.second;
+ inputTypeMissable = unnestTypeResult.third;
+ }
+ }
+
+ List<IAType> projectTypeList = new ArrayList<>(projectPathList.size());
+ for (List<String> projectPath : projectPathList) {
+ IAType projectTypePrime;
+ boolean projectTypeNullable, projectTypeMissable;
+ if (projectPath == null) {
+ projectTypePrime = inputTypePrime;
+ projectTypeNullable = inputTypeNullable;
+ projectTypeMissable = inputTypeMissable;
+ } else if (inputTypePrime == null ||
+ // handle special case of the empty field name in
+ // ExternalIndexingOperations.FILE_INDEX_FIELD_NAMES
+ (projectPath.size() == 1 && projectPath.get(0).isEmpty())) {
+ projectTypePrime = null; // ANY
+ projectTypeNullable = projectTypeMissable = true;
+ } else {
+ if (inputTypePrime.getTypeTag() != ATypeTag.OBJECT) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+ }
+ Triple<IAType, Boolean, Boolean> projectTypeResult =
+ KeyFieldTypeUtil.getKeyProjectType((ARecordType) inputTypePrime, projectPath, null);
+ if (projectTypeResult == null) {
+ throw new AsterixException(ErrorCode.METADATA_ERROR, projectPath.toString());
+ }
+ projectTypePrime = projectTypeResult.first;
+ projectTypeNullable = inputTypeNullable || projectTypeResult.second;
+ projectTypeMissable = inputTypeMissable || projectTypeResult.third;
+ }
+ IAType projectType = projectTypePrime == null ? null
+ : KeyFieldTypeUtil.makeUnknownableType(projectTypePrime, projectTypeNullable,
+ projectTypeMissable);
+
+ projectTypeList.add(projectType);
+ }
+
+ searchKeyType.add(projectTypeList);
+ }
+ isOverridingKeyTypes = false;
+ } else {
+ isOverridingKeyTypes = true;
}
- return new Index(dataverseName, datasetName, indexName, indexStructure, searchKey, keyFieldSourceIndicator,
- searchKeyType, gramLength, isOverridingKeyTypes, isEnforcingKeys, isPrimaryIndex, pendingOp);
+ // create index details structure
+ Index.IIndexDetails indexDetails;
+ switch (Index.IndexCategory.of(indexType)) {
+ case VALUE:
+ List<List<String>> keyFieldNames =
+ searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+ List<IAType> keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+ indexDetails = new Index.ValueIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+ isOverridingKeyTypes);
+ break;
+ case TEXT:
+ keyFieldNames =
+ searchElements.stream().map(Pair::getSecond).map(l -> l.get(0)).collect(Collectors.toList());
+ keyFieldTypes = searchKeyType.stream().map(l -> l.get(0)).collect(Collectors.toList());
+ // Check if there is a gram length as well.
+ int gramLength = -1;
+ int gramLenPos = indexRecord.getType().getFieldIndex(GRAM_LENGTH_FIELD_NAME);
+ if (gramLenPos >= 0) {
+ gramLength = ((AInt32) indexRecord.getValueByPos(gramLenPos)).getIntegerValue();
+ }
+ indexDetails = new Index.TextIndexDetails(keyFieldNames, keyFieldSourceIndicator, keyFieldTypes,
+ isOverridingKeyTypes, gramLength, fullTextConfig);
+ break;
+ case ARRAY:
+ List<Index.ArrayIndexElement> elementList = new ArrayList<>(searchElementCount);
+ for (int i = 0; i < searchElementCount; i++) {
+ Pair<List<List<String>>, List<List<String>>> searchElement = searchElements.get(i);
+ List<IAType> typeList = searchKeyType.get(i);
+ int sourceIndicator = keyFieldSourceIndicator.get(i);
+ elementList.add(new Index.ArrayIndexElement(searchElement.first, searchElement.second, typeList,
+ sourceIndicator));
+ }
+ indexDetails = new Index.ArrayIndexDetails(elementList, isOverridingKeyTypes);
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
+ }
+
+ int isEnforcedFieldPos = indexRecord.getType().getFieldIndex(INDEX_ISENFORCED_FIELD_NAME);
+ Boolean isEnforcingKeys = false;
+ if (isEnforcedFieldPos > 0) {
+ isEnforcingKeys = ((ABoolean) indexRecord.getValueByPos(isEnforcedFieldPos)).getBoolean();
+ }
+ Boolean isPrimaryIndex =
+ ((ABoolean) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
+ .getBoolean();
+ int pendingOp = ((AInt32) indexRecord.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
+ .getIntegerValue();
+
+ return new Index(dataverseName, datasetName, indexName, indexType, indexDetails, isEnforcingKeys,
+ isPrimaryIndex, pendingOp);
}
@Override
- public ITupleReference getTupleFromMetadataEntity(Index index) throws HyracksDataException {
+ public ITupleReference getTupleFromMetadataEntity(Index index) throws HyracksDataException, AlgebricksException {
String dataverseCanonicalName = index.getDataverseName().getCanonicalForm();
// write the key in the first 3 fields of the tuple
@@ -231,15 +461,30 @@
recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_INDEXNAME_FIELD_INDEX, fieldValue);
// write field 3
+ IndexType indexType = index.getIndexType();
fieldValue.reset();
- aString.setValue(index.getIndexType().toString());
+ aString.setValue(indexType.toString());
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_INDEXSTRUCTURE_FIELD_INDEX, fieldValue);
// write field 4
primaryKeyListBuilder.reset((AOrderedListType) MetadataRecordTypes.INDEX_RECORDTYPE
.getFieldTypes()[MetadataRecordTypes.INDEX_ARECORD_SEARCHKEY_FIELD_INDEX]);
- List<List<String>> searchKey = index.getKeyFieldNames();
+ List<List<String>> searchKey;
+ switch (Index.IndexCategory.of(indexType)) {
+ case VALUE:
+ searchKey = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+ break;
+ case TEXT:
+ searchKey = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames();
+ break;
+ case ARRAY:
+ // If we have a complex index, we persist all of the names in the complex SK name array instead.
+ searchKey = Collections.emptyList();
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, indexType.toString());
+ }
for (List<String> field : searchKey) {
listBuilder.reset(stringList);
for (String subField : field) {
@@ -290,14 +535,88 @@
/**
* Keep protected to allow other extensions to add additional fields
*/
- protected void writeOpenFields(Index index) throws HyracksDataException {
- writeGramLength(index);
+ protected void writeOpenFields(Index index) throws HyracksDataException, AlgebricksException {
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case TEXT:
+ Index.TextIndexDetails textIndexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ writeGramLength(textIndexDetails);
+ writeFullTextConfig(textIndexDetails);
+ break;
+ case ARRAY:
+ writeComplexSearchKeys((Index.ArrayIndexDetails) index.getIndexDetails());
+ break;
+ }
writeSearchKeyType(index);
writeEnforced(index);
writeSearchKeySourceIndicator(index);
}
- private void writeGramLength(Index index) throws HyracksDataException {
+ private void writeComplexSearchKeys(Index.ArrayIndexDetails indexDetails) throws HyracksDataException {
+ complexSearchKeyNameListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ for (Index.ArrayIndexElement element : indexDetails.getElementList()) {
+ if (element.getUnnestList().isEmpty()) {
+ // If this is not a complex search key, write the field names as before.
+ buildSearchKeyNameList(element.getProjectList());
+ itemValue.reset();
+ listBuilder.write(itemValue.getDataOutput(), true);
+ } else {
+ // Otherwise, we create a complex searchkey name record.
+ complexSearchKeyNameRecordBuilder.reset(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+
+ nameValue.reset();
+ aString.setValue(COMPLEXSEARCHKEY_UNNEST_FIELD_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ buildSearchKeyNameList(element.getUnnestList());
+ itemValue.reset();
+ listBuilder.write(itemValue.getDataOutput(), true);
+ complexSearchKeyNameRecordBuilder.addField(nameValue, itemValue);
+
+ if (element.getProjectList().get(0) != null) {
+ nameValue.reset();
+ aString.setValue(COMPLEXSEARCHKEY_PROJECT_FIELD_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ buildSearchKeyNameList(element.getProjectList());
+ itemValue.reset();
+ listBuilder.write(itemValue.getDataOutput(), true);
+ complexSearchKeyNameRecordBuilder.addField(nameValue, itemValue);
+ }
+
+ itemValue.reset();
+ complexSearchKeyNameRecordBuilder.write(itemValue.getDataOutput(), true);
+ }
+ complexSearchKeyNameListBuilder.addItem(itemValue);
+ }
+
+ nameValue.reset();
+ fieldValue.reset();
+ aString.setValue(INDEX_SEARCHKEY_ELEMENTS_FIELD_NAME);
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+ complexSearchKeyNameListBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+
+ private void buildSearchKeyNameList(List<List<String>> fieldList) throws HyracksDataException {
+ listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ for (List<String> nestedField : fieldList) {
+ if (nestedField == null) {
+ itemValue.reset();
+ nullSerde.serialize(ANull.NULL, itemValue.getDataOutput());
+ } else {
+ innerListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ for (String subField : nestedField) {
+ itemValue.reset();
+ aString.setValue(subField);
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ innerListBuilder.addItem(itemValue);
+ }
+ itemValue.reset();
+ innerListBuilder.write(itemValue.getDataOutput(), true);
+ }
+ listBuilder.addItem(itemValue);
+ }
+ }
+
+ private void writeGramLength(Index.TextIndexDetails index) throws HyracksDataException {
if (index.getGramLength() > 0) {
fieldValue.reset();
nameValue.reset();
@@ -308,28 +627,75 @@
}
}
- private void writeSearchKeyType(Index index) throws HyracksDataException {
- if (index.isOverridingKeyFieldTypes()) {
- OrderedListBuilder typeListBuilder = new OrderedListBuilder();
- typeListBuilder.reset(new AOrderedListType(BuiltinType.ANY, null));
+ private void writeFullTextConfig(Index.TextIndexDetails index) throws HyracksDataException {
+ if (!Strings.isNullOrEmpty(index.getFullTextConfigName())) {
nameValue.reset();
- aString.setValue(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
-
+ aString.setValue(FULL_TEXT_CONFIG_FIELD_NAME);
stringSerde.serialize(aString, nameValue.getDataOutput());
- List<IAType> searchKeyType = index.getKeyFieldTypes();
- for (IAType type : searchKeyType) {
- itemValue.reset();
- aString.setValue(type.getTypeName());
- stringSerde.serialize(aString, itemValue.getDataOutput());
- typeListBuilder.addItem(itemValue);
- }
fieldValue.reset();
- typeListBuilder.write(fieldValue.getDataOutput(), true);
+ aString.setValue(index.getFullTextConfigName());
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+
recordBuilder.addField(nameValue, fieldValue);
}
}
+ private void writeSearchKeyType(Index index) throws HyracksDataException, AlgebricksException {
+ if (!index.getIndexDetails().isOverridingKeyFieldTypes()) {
+ return;
+ }
+
+ OrderedListBuilder typeListBuilder = new OrderedListBuilder();
+ typeListBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+
+ nameValue.reset();
+ aString.setValue(INDEX_SEARCHKEY_TYPE_FIELD_NAME);
+
+ stringSerde.serialize(aString, nameValue.getDataOutput());
+
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ // For value and text indexes, we persist the type as a single string (backwards compatibility).
+ case VALUE:
+ for (IAType type : ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
+ itemValue.reset();
+ aString.setValue(type.getTypeName());
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ typeListBuilder.addItem(itemValue);
+ }
+ break;
+ case TEXT:
+ for (IAType type : ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldTypes()) {
+ itemValue.reset();
+ aString.setValue(type.getTypeName());
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ typeListBuilder.addItem(itemValue);
+ }
+ break;
+ case ARRAY:
+ // For array indexes we persist the type as a list of strings.
+ for (Index.ArrayIndexElement element : ((Index.ArrayIndexDetails) index.getIndexDetails())
+ .getElementList()) {
+ listBuilder.reset(AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE);
+ for (IAType type : element.getTypeList()) {
+ itemValue.reset();
+ aString.setValue(type.getTypeName());
+ stringSerde.serialize(aString, itemValue.getDataOutput());
+ listBuilder.addItem(itemValue);
+ }
+ itemValue.reset();
+ listBuilder.write(itemValue.getDataOutput(), true);
+ typeListBuilder.addItem(itemValue);
+ }
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
+ }
+ fieldValue.reset();
+ typeListBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(nameValue, fieldValue);
+ }
+
private void writeEnforced(Index index) throws HyracksDataException {
if (index.isEnforced()) {
fieldValue.reset();
@@ -341,8 +707,22 @@
}
}
- private void writeSearchKeySourceIndicator(Index index) throws HyracksDataException {
- List<Integer> keySourceIndicator = index.getKeyFieldSourceIndicators();
+ private void writeSearchKeySourceIndicator(Index index) throws HyracksDataException, AlgebricksException {
+ List<Integer> keySourceIndicator;
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case VALUE:
+ keySourceIndicator = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+ break;
+ case TEXT:
+ keySourceIndicator = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators();
+ break;
+ case ARRAY:
+ keySourceIndicator = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+ .map(Index.ArrayIndexElement::getSourceIndicator).collect(Collectors.toList());
+ break;
+ default:
+ throw new AsterixException(ErrorCode.METADATA_ERROR, index.getIndexType().toString());
+ }
boolean needSerialization = false;
if (keySourceIndicator != null) {
for (int source : keySourceIndicator) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
index 1079904..c2a9ee3 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/MetadataTupleTranslatorProvider.java
@@ -65,6 +65,14 @@
return new FunctionTupleTranslator(txnId, metadataNode, getTuple);
}
+ public FullTextConfigMetadataEntityTupleTranslator getFullTextConfigTupleTranslator(boolean getTuple) {
+ return new FullTextConfigMetadataEntityTupleTranslator(getTuple);
+ }
+
+ public FullTextFilterMetadataEntityTupleTranslator getFullTextFilterTupleTranslator(boolean getTuple) {
+ return new FullTextFilterMetadataEntityTupleTranslator(getTuple);
+ }
+
public IndexTupleTranslator getIndexTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
return new IndexTupleTranslator(txnId, metadataNode, getTuple);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
index 261687b..5e44e32 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockKey.java
@@ -34,6 +34,8 @@
EXTENSION,
FEED_POLICY,
FUNCTION,
+ FULL_TEXT_CONFIG,
+ FULL_TEXT_FILTER,
LIBRARY,
ADAPTER,
MERGE_POLICY,
@@ -108,6 +110,14 @@
return new MetadataLockKey(EntityKind.FUNCTION, null, dataverseName, functionName);
}
+ static MetadataLockKey createFullTextConfigLockKey(DataverseName dataverseName, String fullTextConfigName) {
+ return new MetadataLockKey(EntityKind.FULL_TEXT_CONFIG, null, dataverseName, fullTextConfigName);
+ }
+
+ static MetadataLockKey createFullTextFilterLockKey(DataverseName dataverseName, String fullTextFilterName) {
+ return new MetadataLockKey(EntityKind.FULL_TEXT_FILTER, null, dataverseName, fullTextFilterName);
+ }
+
static MetadataLockKey createLibraryLockKey(DataverseName dataverseName, String libraryName) {
return new MetadataLockKey(EntityKind.LIBRARY, null, dataverseName, libraryName);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
index 63d01d1..48508e1 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/lock/MetadataLockManager.java
@@ -110,6 +110,38 @@
}
@Override
+ public void acquireFullTextConfigReadLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+ throws AlgebricksException {
+ MetadataLockKey key = MetadataLockKey.createFullTextConfigLockKey(dataverseName, fullTextConfigName);
+ IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+ locks.add(IMetadataLock.Mode.READ, lock);
+ }
+
+ @Override
+ public void acquireFullTextConfigWriteLock(LockList locks, DataverseName dataverseName, String fullTextConfigName)
+ throws AlgebricksException {
+ MetadataLockKey key = MetadataLockKey.createFullTextConfigLockKey(dataverseName, fullTextConfigName);
+ IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+ locks.add(IMetadataLock.Mode.WRITE, lock);
+ }
+
+ @Override
+ public void acquireFullTextFilterReadLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+ throws AlgebricksException {
+ MetadataLockKey key = MetadataLockKey.createFullTextFilterLockKey(dataverseName, fullTextFilterName);
+ IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+ locks.add(IMetadataLock.Mode.READ, lock);
+ }
+
+ @Override
+ public void acquireFullTextFilterWriteLock(LockList locks, DataverseName dataverseName, String fullTextFilterName)
+ throws AlgebricksException {
+ MetadataLockKey key = MetadataLockKey.createFullTextFilterLockKey(dataverseName, fullTextFilterName);
+ IMetadataLock lock = mdlocks.computeIfAbsent(key, LOCK_FUNCTION);
+ locks.add(IMetadataLock.Mode.WRITE, lock);
+ }
+
+ @Override
public void acquireLibraryReadLock(LockList locks, DataverseName dataverseName, String libraryName)
throws AlgebricksException {
MetadataLockKey key = MetadataLockKey.createLibraryLockKey(dataverseName, libraryName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
new file mode 100644
index 0000000..f4dfe56
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
@@ -0,0 +1,380 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.metadata.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AUnorderedListType;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+public class ArrayIndexUtil {
+ /**
+ * @deprecated Use the project + unnest scheme instead of array indicators.
+ */
+ public static IAType getSubFieldInArrayType(ARecordType recordType, List<String> subFieldName,
+ List<Integer> arrayDepthIndicators) throws AlgebricksException {
+ IAType subType = recordType.getFieldType(subFieldName.get(0));
+ for (int i = 1; i < subFieldName.size(); i++) {
+ if (subType == null) {
+ return null;
+ } else if (subType.getTypeTag().equals(ATypeTag.UNION)) {
+ // Support enforced types here.
+ subType = ((AUnionType) subType).getActualType();
+ if (!subType.getTypeTag().equals(ATypeTag.OBJECT) && !subType.getTypeTag().equals(ATypeTag.ARRAY)
+ && !subType.getTypeTag().equals(ATypeTag.MULTISET)) {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "Field accessor is not defined for values of type " + subType.getTypeTag());
+ }
+ }
+ if (subType.getTypeTag().equals(ATypeTag.OBJECT) && arrayDepthIndicators.get(i - 1) == 0) {
+ subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
+ } else if ((subType.getTypeTag().equals(ATypeTag.ARRAY) || subType.getTypeTag().equals(ATypeTag.MULTISET))
+ && arrayDepthIndicators.get(i - 1) > 0) {
+ for (int j = 0; j < arrayDepthIndicators.get(i - 1); j++) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ }
+ subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+ } else {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ (arrayDepthIndicators.get(i - 1) > 0)
+ ? "Object type given, but array depth indicator is " + "non-zero."
+ : "Array/multiset type given, but array depth indicator is zero.");
+ }
+ }
+ if (subType != null && arrayDepthIndicators.get(arrayDepthIndicators.size() - 1) > 0) {
+ // If the end field is an array, we must extract the list item here as well.
+ for (int j = 0; j < arrayDepthIndicators.get(arrayDepthIndicators.size() - 1); j++) {
+ if (subType instanceof AbstractCollectionType) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ } else {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "Array type expected for last term, but given: "
+ + ((subType != null) ? subType.getTypeTag() : "null"));
+ }
+ }
+ }
+ return subType;
+ }
+
+ /**
+ * Given a path of complex types (i.e. lists + records), determine the nullability of the field.
+ * @deprecated Use the project + unnest scheme instead of array indicators.
+ */
+ public static boolean isSubFieldNullable(ARecordType recordType, List<String> subFieldName,
+ List<Integer> arrayIndicators) throws AlgebricksException {
+ IAType subType = recordType.getFieldType(subFieldName.get(0));
+ for (int i = 1; i < subFieldName.size(); i++) {
+ if (subType == null) {
+ return true;
+ }
+ if (subType.getTypeTag().equals(ATypeTag.UNION)) {
+ if (NonTaggedFormatUtil.isOptional(subType)) {
+ return true;
+ }
+ subType = ((AUnionType) subType).getActualType();
+ if (subType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new AsterixException(
+ "Field accessor is not defined for values of type " + subType.getTypeTag());
+ }
+ }
+
+ if (subType instanceof ARecordType) {
+ subType = ((ARecordType) subType).getFieldType(subFieldName.get(i));
+ } else if (subType instanceof AbstractCollectionType && arrayIndicators.get(i - 1) > 0) {
+ for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ }
+ subType = (subType != null) ? ((ARecordType) subType).getFieldType(subFieldName.get(i)) : null;
+ } else {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Illegal field type " + subType.getTypeTag() + " when checking field nullability");
+ }
+ }
+ return subType == null || NonTaggedFormatUtil.isOptional(subType);
+ }
+
+ /**
+ * Similar function to Index's "getNonNullableOpenFieldType", but accounts for array fields as well.
+ * @deprecated Use the project + unnest scheme instead of array indicators.
+ */
+ public static Pair<IAType, Boolean> getNonNullableOpenFieldType(IAType fieldType, List<String> fieldName,
+ ARecordType recType, List<Integer> arrayIndicators) throws AlgebricksException {
+ Pair<IAType, Boolean> keyPairType = null;
+ IAType subType = recType;
+ boolean nullable = false;
+ for (int i = 0; i < fieldName.size(); i++) {
+ if (subType instanceof AUnionType) {
+ nullable = nullable || ((AUnionType) subType).isUnknownableType();
+ subType = ((AUnionType) subType).getActualType();
+ }
+ if (subType instanceof ARecordType) {
+ subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+ } else if ((subType instanceof AOrderedListType || subType instanceof AUnorderedListType)
+ && arrayIndicators.get(i - 1) > 0) {
+ for (int j = 0; j < arrayIndicators.get(i - 1); j++) {
+ subType = TypeComputeUtils.extractListItemType(subType);
+ }
+ if (subType instanceof ARecordType) {
+ subType = ((ARecordType) subType).getFieldType(fieldName.get(i));
+ } else {
+ throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+ }
+ } else {
+ throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+ }
+
+ if (subType == null) {
+ keyPairType = Index.getNonNullableType(fieldType);
+ break;
+ }
+ }
+ if (subType != null) {
+ IAType keyType = ArrayIndexUtil.getSubFieldInArrayType(recType, fieldName, arrayIndicators);
+ Pair<IAType, Boolean> pair = Index.getNonNullableType(keyType);
+ pair.second = pair.second || ArrayIndexUtil.isSubFieldNullable(recType, fieldName, arrayIndicators);
+ keyPairType = pair;
+ }
+ keyPairType.second = keyPairType.second || nullable;
+ return keyPairType;
+ }
+
+ /**
+ * @deprecated Use new unnestList and projectList scheme.
+ */
+ public static List<String> getFlattenedKeyFieldNames(List<List<String>> unnestList, List<String> projectList) {
+ if (unnestList == null) {
+ return projectList;
+
+ } else {
+ List<String> flattenedKeyNameList = new ArrayList<>();
+ for (List<String> unnestField : unnestList) {
+ flattenedKeyNameList.addAll(unnestField);
+ }
+ if (projectList != null) {
+ flattenedKeyNameList.addAll(projectList);
+ }
+ return flattenedKeyNameList;
+ }
+ }
+
+ /**
+ * @deprecated Use new unnestList and projectList scheme.
+ */
+ public static List<Integer> getArrayDepthIndicator(List<List<String>> unnestList, List<String> projectList) {
+ if (unnestList == null) {
+ // A simple element has a flat set of depth indicators.
+ List<Integer> depthIndicator = new ArrayList<>();
+ for (String ignored : projectList) {
+ depthIndicator.add(0);
+ }
+ return depthIndicator;
+
+ } else {
+ List<Integer> depthIndicatorPrefix = new ArrayList<>();
+ for (List<String> unnestField : unnestList) {
+ for (int i = 0; i < unnestField.size() - 1; i++) {
+ depthIndicatorPrefix.add(0);
+ }
+ depthIndicatorPrefix.add(1);
+ }
+
+ if (projectList == null) {
+ // Stop here. The prefix is the indicator itself.
+ return depthIndicatorPrefix;
+
+ } else {
+ List<Integer> depthIndicator = new ArrayList<>(depthIndicatorPrefix);
+ for (int i = 0; i < projectList.size(); i++) {
+ depthIndicator.add(0);
+ }
+ return depthIndicator;
+ }
+ }
+ }
+
+ /**
+ * @deprecated Use new unnestList and projectList scheme.
+ * @return The record paths and non-zero depth indicators associated each record of fields from an array index.
+ */
+ public static Pair<List<List<String>>, List<Integer>> unnestComplexRecordPath(List<String> fieldName,
+ List<Integer> depthIndicators) {
+ List<List<String>> resultantPaths = new ArrayList<>();
+ List<Integer> resultantArrayIndicators = new ArrayList<>();
+ List<String> workingRecordPath = new ArrayList<>();
+ for (int i = 0; i < depthIndicators.size(); i++) {
+ workingRecordPath.add(fieldName.get(i));
+
+ if (i == depthIndicators.size() - 1 || depthIndicators.get(i) > 0) {
+ resultantArrayIndicators.add(depthIndicators.get(i));
+ resultantPaths.add(workingRecordPath);
+ workingRecordPath = new ArrayList<>();
+ }
+ }
+ return new Pair<>(resultantPaths, resultantArrayIndicators);
+ }
+
+ /**
+ * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
+ * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
+ * distinct record path and invoke the appropriate commands for each scenario.
+ * <p>
+ * Here, we keep track of the record/list type at each step and give this to each command.
+ */
+ public static void walkArrayPath(ARecordType baseRecordType, List<String> flattenedFieldName,
+ List<Integer> flattenedDepthIndicators, TypeTrackerCommandExecutor commandExecutor)
+ throws AlgebricksException {
+ ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+ List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
+ List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+
+ // If we are given no base record type, then we do not need to keep track of the record type. We are solely
+ // using this walk for its flags.
+ boolean isTrackingType = baseRecordType != null;
+
+ IAType workingType = baseRecordType;
+ for (int i = 0; i < fieldNamesPerArray.size(); i++) {
+ ARecordType startingStepRecordType = null;
+ if (isTrackingType) {
+ if (!workingType.getTypeTag().equals(ATypeTag.OBJECT)) {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Mismatched record type to depth-"
+ + "indicators. Expected record type, but got: " + workingType.getTypeTag());
+ }
+ startingStepRecordType = (ARecordType) workingType;
+ workingType = Index.getNonNullableOpenFieldType(
+ startingStepRecordType.getSubFieldType(fieldNamesPerArray.get(i)), fieldNamesPerArray.get(i),
+ startingStepRecordType).first;
+ }
+
+ for (int j = 0; j < depthOfArraySteps.get(i); j++) {
+ if (isTrackingType) {
+ workingType = TypeComputeUtils.extractListItemType(workingType);
+ if (workingType == null) {
+ throw new AsterixException(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Expected list type inside record: " + startingStepRecordType);
+ }
+ }
+ boolean isFirstArrayStep = i == 0;
+ boolean isFirstUnnestInStep = j == 0;
+ boolean isLastUnnestInIntermediateStep =
+ j == depthOfArraySteps.get(i) - 1 && i < fieldNamesPerArray.size() - 1;
+ commandExecutor.executeActionOnEachArrayStep(startingStepRecordType, workingType,
+ fieldNamesPerArray.get(i), isFirstArrayStep, isFirstUnnestInStep,
+ isLastUnnestInIntermediateStep);
+ }
+
+ if (i == fieldNamesPerArray.size() - 1) {
+ boolean requiresOnlyOneUnnest = depthOfArraySteps.stream().reduce(0, Integer::sum).equals(1);
+ boolean isNonArrayStep = depthOfArraySteps.get(i) == 0;
+ commandExecutor.executeActionOnFinalArrayStep(startingStepRecordType, fieldNamesPerArray.get(i),
+ isNonArrayStep, requiresOnlyOneUnnest);
+ }
+ }
+ }
+
+ /**
+ * Given the {@code Index}'s representation of an array path (i.e. a concatenation of record paths, with array
+ * steps specified in depths corresponding to an index in the aforementioned record path array), traverse each
+ * distinct record path and invoke the appropriate commands for each scenario.
+ * <p>
+ * Here, we keep track of the total number of actions performed and give this to each command.
+ */
+ public static void walkArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators,
+ ActionCounterCommandExecutor commandExecutor) throws AlgebricksException {
+ ArrayPath arrayPath = new ArrayPath(flattenedFieldName, flattenedDepthIndicators).invoke();
+ List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
+ List<Integer> depthOfArraySteps = arrayPath.depthOfArraySteps;
+
+ int numberOfActionsPerformed = 0;
+ for (int i = 0; i < fieldNamesPerArray.size(); i++) {
+ int unnestLevel = depthOfArraySteps.get(i);
+ if (i == 0) {
+ commandExecutor.executeActionOnFirstArrayStep();
+ numberOfActionsPerformed++;
+ unnestLevel--;
+ }
+
+ for (int j = 0; j < unnestLevel; j++) {
+ commandExecutor.executeActionOnIntermediateArrayStep(numberOfActionsPerformed++);
+ }
+
+ if (i == fieldNamesPerArray.size() - 1) {
+ commandExecutor.executeActionOnFinalArrayStep(numberOfActionsPerformed++);
+ }
+ }
+ }
+
+ public interface ActionCounterCommandExecutor {
+ void executeActionOnFirstArrayStep() throws AlgebricksException;
+
+ void executeActionOnIntermediateArrayStep(int numberOfActionsAlreadyPerformed) throws AlgebricksException;
+
+ void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) throws AlgebricksException;
+ }
+
+ public interface TypeTrackerCommandExecutor {
+ void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+ List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+ boolean isLastUnnestInIntermediateStep) throws AlgebricksException;
+
+ void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+ boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException;
+ }
+
+ private static class ArrayPath {
+ private final List<String> flattenedFieldName;
+ private final List<Integer> flattenedDepthIndicators;
+ private List<List<String>> fieldNamesPerArray;
+ private List<Integer> depthOfArraySteps;
+
+ public ArrayPath(List<String> flattenedFieldName, List<Integer> flattenedDepthIndicators) {
+ this.flattenedFieldName = flattenedFieldName;
+ this.flattenedDepthIndicators = flattenedDepthIndicators;
+ }
+
+ public ArrayPath invoke() {
+ fieldNamesPerArray = new ArrayList<>();
+ depthOfArraySteps = new ArrayList<>();
+ List<String> workingRecordPath = new ArrayList<>();
+ for (int i = 0; i < flattenedDepthIndicators.size(); i++) {
+ workingRecordPath.add(flattenedFieldName.get(i));
+
+ if (i == flattenedDepthIndicators.size() - 1 || flattenedDepthIndicators.get(i) > 0) {
+ depthOfArraySteps.add(flattenedDepthIndicators.get(i));
+ fieldNamesPerArray.add(workingRecordPath);
+ workingRecordPath = new ArrayList<>();
+ }
+ }
+ return this;
+ }
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index 9847fe4..b899e16 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -80,6 +80,11 @@
public class ExternalIndexingOperations {
private static final Logger LOGGER = LogManager.getLogger();
+ /**
+ * Note: there's a special handling of this empty indexing field name in
+ * {@link org.apache.asterix.metadata.entitytupletranslators.IndexTupleTranslator
+ * IndexTupleTranslator.createMetadataEntityFromARecord()}
+ */
public static final List<List<String>> FILE_INDEX_FIELD_NAMES =
Collections.unmodifiableList(Collections.singletonList(Collections.singletonList("")));
public static final List<IAType> FILE_INDEX_FIELD_TYPES =
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java
new file mode 100644
index 0000000..563bab4
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/FullTextUtil.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.metadata.utils;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.runtime.fulltext.AbstractFullTextFilterDescriptor;
+import org.apache.asterix.runtime.fulltext.FullTextConfigDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+
+import com.google.common.collect.ImmutableList;
+
+public class FullTextUtil {
+ public static IFullTextConfigEvaluatorFactory fetchFilterAndCreateConfigEvaluator(MetadataProvider metadataProvider,
+ DataverseName dataverseName, String configName) throws AlgebricksException {
+ FullTextConfigDescriptor configDescriptor =
+ metadataProvider.findFullTextConfig(dataverseName, configName).getFullTextConfig();
+
+ ImmutableList.Builder<AbstractFullTextFilterDescriptor> filterDescriptorsBuilder = ImmutableList.builder();
+ for (String filterName : configDescriptor.getFilterNames()) {
+ filterDescriptorsBuilder
+ .add(metadataProvider.findFullTextFilter(dataverseName, filterName).getFullTextFilter());
+ }
+
+ return configDescriptor.createEvaluatorFactory(filterDescriptorsBuilder.build());
+ }
+
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
index 311e4b6..f5b2697 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/IndexUtil.java
@@ -67,8 +67,16 @@
if (index.isPrimaryIndex()) {
return DatasetUtil.createBTreeFieldsWhenThereisAFilter(dataset);
}
+ int numSecondaryKeys;
+ if (index.getIndexType() == DatasetConfig.IndexType.BTREE) {
+ numSecondaryKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
+ } else if (index.getIndexType() == DatasetConfig.IndexType.ARRAY) {
+ numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+ .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE, index.getIndexType().toString());
+ }
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
int[] btreeFields = new int[numSecondaryKeys + numPrimaryKeys];
for (int k = 0; k < btreeFields.length; k++) {
btreeFields[k] = k;
@@ -82,9 +90,14 @@
return empty;
}
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys;
switch (index.getIndexType()) {
+ case ARRAY:
+ numSecondaryKeys = ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+ .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ return new int[] { numPrimaryKeys + numSecondaryKeys };
case BTREE:
+ numSecondaryKeys = ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
return new int[] { numPrimaryKeys + numSecondaryKeys };
case RTREE:
case LENGTH_PARTITIONED_NGRAM_INVIX:
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
index d20ff53..8b69652 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/InvertedIndexResourceFactoryProvider.java
@@ -51,6 +51,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexLocalResourceFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -68,7 +69,8 @@
IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
// Get basic info
List<List<String>> primaryKeys = dataset.getPrimaryKeys();
- List<List<String>> secondaryKeys = index.getKeyFieldNames();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ List<List<String>> secondaryKeys = indexDetails.getKeyFieldNames();
List<String> filterFieldName = DatasetUtil.getFilterField(dataset);
int numPrimaryKeys = primaryKeys.size();
int numSecondaryKeys = secondaryKeys.size();
@@ -123,12 +125,17 @@
IBinaryComparatorFactory[] tokenCmpFactories =
getTokenComparatorFactories(dataset, index, recordType, metaType);
IBinaryTokenizerFactory tokenizerFactory = getTokenizerFactory(dataset, index, recordType, metaType);
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+ FullTextUtil.fetchFilterAndCreateConfigEvaluator(mdProvider, index.getDataverseName(),
+ indexDetails.getFullTextConfigName());
+
return new LSMInvertedIndexLocalResourceFactory(storageManager, typeTraits, cmpFactories, filterTypeTraits,
filterCmpFactories, secondaryFilterFields, opTrackerFactory, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, true, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- isPartitioned, invertedIndexFields, secondaryFilterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+ fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
+ secondaryFilterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
+ bloomFilterFalsePositiveRate);
}
// Returns an array of the type traits of the inverted list elements
@@ -152,7 +159,8 @@
private static ITypeTraits[] getTokenTypeTraits(Dataset dataset, Index index, ARecordType recordType,
ARecordType metaType) throws AlgebricksException {
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IndexType indexType = index.getIndexType();
// Sanity checks.
if (numPrimaryKeys > 1) {
@@ -166,14 +174,14 @@
boolean isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
|| indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
@@ -188,7 +196,8 @@
private static IBinaryComparatorFactory[] getTokenComparatorFactories(Dataset dataset, Index index,
ARecordType recordType, ARecordType metaType) throws AlgebricksException {
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IndexType indexType = index.getIndexType();
// Sanity checks.
if (numPrimaryKeys > 1) {
@@ -201,15 +210,15 @@
}
boolean isPartitioned = indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
|| indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
ARecordType sourceType;
if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
// Comparators and type traits for tokens.
int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
@@ -225,7 +234,8 @@
private static IBinaryTokenizerFactory getTokenizerFactory(Dataset dataset, Index index, ARecordType recordType,
ARecordType metaType) throws AlgebricksException {
int numPrimaryKeys = dataset.getPrimaryKeys().size();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IndexType indexType = index.getIndexType();
// Sanity checks.
if (numPrimaryKeys > 1) {
@@ -237,19 +247,19 @@
indexType, 1);
}
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), sourceType);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ indexDetails.getKeyFieldNames().get(0), sourceType);
IAType secondaryKeyType = keyTypePair.first;
// Set tokenizer factory.
// TODO: We might want to expose the hashing option at the AQL level,
// and add the choice to the index metadata.
return NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
- index.getGramLength());
+ indexDetails.getGramLength());
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
index e074241..b2026d7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/KeyFieldTypeUtil.java
@@ -20,6 +20,7 @@
import java.util.ArrayList;
import java.util.List;
+import java.util.Objects;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.exceptions.CompilationException;
@@ -27,11 +28,18 @@
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.asterix.om.utils.RecordUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.api.exceptions.SourceLocation;
public class KeyFieldTypeUtil {
@@ -92,6 +100,8 @@
* record.
* @return a list of IATypes, one for each corresponding index key field.
* @throws AlgebricksException
+ *
+ * @deprecated use {@link #getKeyProjectType(ARecordType, List, SourceLocation)}
*/
public static List<IAType> getKeyTypes(ARecordType recordType, ARecordType metaRecordType, List<List<String>> keys,
List<Integer> keySourceIndicators) throws AlgebricksException {
@@ -119,17 +129,40 @@
*/
public static List<IAType> getBTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
throws AlgebricksException {
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
List<IAType> indexKeyTypes = new ArrayList<>();
- for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), chooseSource(keySourceIndicators, i, recordType, metaRecordType));
+ for (int i = 0; i < indexDetails.getKeyFieldNames().size(); i++) {
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i),
+ chooseSource(keySourceIndicators, i, recordType, metaRecordType));
indexKeyTypes.add(keyPairType.first);
}
return indexKeyTypes;
}
/**
+ * @see KeyFieldTypeUtil#getBTreeIndexKeyTypes(Index, ARecordType, ARecordType)
+ */
+ public static List<IAType> getArrayBTreeIndexKeyTypes(Index index, ARecordType recordType,
+ ARecordType metaRecordType) throws AlgebricksException {
+ Index.ArrayIndexDetails indexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ List<IAType> indexKeyTypes = new ArrayList<>();
+ for (Index.ArrayIndexElement e : indexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ ARecordType sourceType =
+ (e.getSourceIndicator() == Index.RECORD_INDICATOR) ? recordType : metaRecordType;
+ Pair<IAType, Boolean> keyPairType = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
+ sourceType,
+ ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+ indexKeyTypes.add(keyPairType.first);
+ }
+ }
+ return indexKeyTypes;
+ }
+
+ /**
* Get the types of RTree index key fields
*
* @param index,
@@ -143,11 +176,12 @@
*/
public static List<IAType> getRTreeIndexKeyTypes(Index index, ARecordType recordType, ARecordType metaRecordType)
throws AlgebricksException {
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
List<IAType> indexKeyTypes = new ArrayList<>();
ARecordType targetRecType = chooseSource(keySourceIndicators, 0, recordType, metaRecordType);
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), targetRecType);
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ indexDetails.getKeyFieldNames().get(0), targetRecType);
IAType keyType = keyPairType.first;
IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
int numKeys = KeyFieldTypeUtil.getNumSecondaryKeys(index, targetRecType, metaRecordType);
@@ -171,17 +205,22 @@
*/
public static int getNumSecondaryKeys(Index index, ARecordType recordType, ARecordType metaRecordType)
throws AlgebricksException {
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
switch (index.getIndexType()) {
+ case ARRAY:
+ return ((Index.ArrayIndexDetails) index.getIndexDetails()).getElementList().stream()
+ .map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
case BTREE:
+ return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
case SINGLE_PARTITION_WORD_INVIX:
case SINGLE_PARTITION_NGRAM_INVIX:
case LENGTH_PARTITIONED_WORD_INVIX:
case LENGTH_PARTITIONED_NGRAM_INVIX:
- return index.getKeyFieldNames().size();
+ return ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
case RTREE:
- Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0),
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
+ Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0),
chooseSource(keySourceIndicators, 0, recordType, metaRecordType));
IAType keyType = keyPairType.first;
return NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag()) * 2;
@@ -208,4 +247,107 @@
ARecordType metaRecordType) {
return keySourceIndicators.get(index) == 0 ? recordType : metaRecordType;
}
+
+ /**
+ * Returns type after applying UNNEST steps defined by an index element.
+ *
+ * @return { primeType, nullable, missable } or {@code null} if the path is not found in an open record
+ * @throws CompilationException
+ * if path is not found in a closed record
+ */
+ public static Triple<IAType, Boolean, Boolean> getKeyUnnestType(final ARecordType inputType,
+ List<List<String>> unnestPathList, SourceLocation sourceLoc) throws CompilationException {
+ if (unnestPathList.isEmpty()) {
+ return new Triple<>(inputType, false, false);
+ }
+ IAType itemType = inputType;
+ boolean itemTypeNullable = false, itemTypeMissable = false;
+ for (List<String> unnestPath : unnestPathList) {
+ // check that the type is a record at this point
+ if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+ itemType.getTypeTag());
+ }
+ ARecordType itemRecordType = (ARecordType) itemType;
+ Triple<IAType, Boolean, Boolean> fieldTypeResult = getKeyProjectType(itemRecordType, unnestPath, sourceLoc);
+ if (fieldTypeResult == null) {
+ return null;
+ }
+ IAType fieldType = fieldTypeResult.first;
+ boolean fieldTypeNullable = fieldTypeResult.second;
+ boolean fieldTypeMissable = fieldTypeResult.third;
+ // check that we've arrived to a collection type
+ if (!fieldType.getTypeTag().isListType()) {
+ throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC,
+ sourceLoc, ExceptionUtil.toExpectedTypeString(new byte[] {
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG }),
+ fieldType);
+ }
+ AbstractCollectionType fieldListType = (AbstractCollectionType) fieldType;
+ IAType fieldListItemType = fieldListType.getItemType();
+ boolean fieldListItemTypeNullable = false, fieldListItemTypeMissable = false;
+ if (fieldListItemType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType fieldListItemTypeUnion = (AUnionType) fieldListItemType;
+ fieldListItemType = fieldListItemTypeUnion.getActualType();
+ fieldListItemTypeNullable = fieldListItemTypeUnion.isNullableType();
+ fieldListItemTypeMissable = fieldListItemTypeUnion.isMissableType();
+ }
+ itemType = fieldListItemType;
+ itemTypeNullable = itemTypeNullable || fieldTypeNullable || fieldListItemTypeNullable;
+ itemTypeMissable = itemTypeMissable || fieldTypeMissable || fieldListItemTypeMissable;
+ }
+ return new Triple<>(itemType, itemTypeNullable, itemTypeMissable);
+ }
+
+ /**
+ * Returns type after applying SELECT steps defined by an index element.
+ *
+ * @return { primeType, nullable, missable } or {@code null} if the path is not found in an open record
+ * @throws CompilationException
+ * if path is not found in a closed record
+ */
+ public static Triple<IAType, Boolean, Boolean> getKeyProjectType(final ARecordType inputType, List<String> path,
+ SourceLocation sourceLoc) throws CompilationException {
+ IAType itemType = inputType;
+ boolean itemTypeNullable = false, itemTypeMissable = false;
+ for (String step : path) {
+ // check that the type is a record at this point
+ if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new CompilationException(ErrorCode.TYPE_MISMATCH_GENERIC, sourceLoc, ATypeTag.OBJECT,
+ itemType.getTypeTag());
+ }
+ ARecordType itemRecordType = (ARecordType) itemType;
+ IAType fieldType = itemRecordType.getFieldType(step);
+ if (fieldType == null) {
+ if (itemRecordType.isOpen()) {
+ // open record type and we couldn't find the field -> ok.
+ return null;
+ } else {
+ // closed record type and we couldn't find the field -> error.
+ throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND, sourceLoc,
+ RecordUtil.toFullyQualifiedName(path));
+ }
+ }
+ if (fieldType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType fieldTypeUnion = (AUnionType) fieldType;
+ itemType = fieldTypeUnion.getActualType();
+ itemTypeNullable = itemTypeNullable || fieldTypeUnion.isNullableType();
+ itemTypeMissable = itemTypeMissable || fieldTypeUnion.isMissableType();
+ } else {
+ itemType = fieldType;
+ }
+ }
+ return new Triple<>(itemType, itemTypeNullable, itemTypeMissable);
+ }
+
+ public static IAType makeUnknownableType(IAType primeType, boolean nullable, boolean missable) {
+ IAType type = Objects.requireNonNull(primeType);
+ if (nullable) {
+ type = AUnionType.createNullableType(type);
+ }
+ if (missable) {
+ type = AUnionType.createMissableType(type);
+ }
+ return type;
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
index 9240f25..b52cddd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataConstants.java
@@ -54,6 +54,8 @@
public static final String COMPACTION_POLICY_DATASET_NAME = "CompactionPolicy";
public static final String EXTERNAL_FILE_DATASET_NAME = "ExternalFile";
public static final String SYNONYM_DATASET_NAME = "Synonym";
+ public static final String FULL_TEXT_CONFIG_DATASET_NAME = "FullTextConfig";
+ public static final String FULL_TEXT_FILTER_DATASET_NAME = "FullTextFilter";
private MetadataConstants() {
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index 4309c2e..7bb11e5 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -18,6 +18,10 @@
*/
package org.apache.asterix.metadata.utils;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.metadata.DataverseName;
@@ -25,6 +29,9 @@
import org.apache.asterix.common.metadata.LockList;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+
public class MetadataLockUtil implements IMetadataLockUtil {
@Override
@@ -89,9 +96,12 @@
@Override
public void createIndexBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
- String datasetName) throws AlgebricksException {
+ String datasetName, String fullTextConfigName) throws AlgebricksException {
lockMgr.acquireDataverseReadLock(locks, dataverseName);
lockMgr.acquireDatasetCreateIndexLock(locks, dataverseName, datasetName);
+ if (!Strings.isNullOrEmpty(fullTextConfigName)) {
+ lockMgr.acquireFullTextConfigReadLock(locks, dataverseName, fullTextConfigName);
+ }
}
@Override
@@ -157,6 +167,43 @@
}
@Override
+ public void createFullTextFilterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String fullTextFilterName) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireFullTextFilterWriteLock(locks, dataverseName, fullTextFilterName);
+ }
+
+ @Override
+ public void dropFullTextFilterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String fullTextFilterName) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireFullTextFilterWriteLock(locks, dataverseName, fullTextFilterName);
+ }
+
+ @Override
+ public void createFullTextConfigBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String fullTextConfigName, ImmutableList<String> fullTextFilterNames) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireFullTextConfigWriteLock(locks, dataverseName, fullTextConfigName);
+
+ // We should avoid sorting the original list, and the original list is immutable and cannot be sorted anyway
+ List<String> fullTextFilterNamesMutable = new ArrayList<>(fullTextFilterNames);
+
+ // sort the filters to guarantee locks are always fetched in the same order to avoid dead lock between filters
+ Collections.sort(fullTextFilterNamesMutable);
+ for (String filterName : fullTextFilterNamesMutable) {
+ lockMgr.acquireFullTextFilterReadLock(locks, dataverseName, filterName);
+ }
+ }
+
+ @Override
+ public void dropFullTextConfigBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
+ String configName) throws AlgebricksException {
+ lockMgr.acquireDataverseReadLock(locks, dataverseName);
+ lockMgr.acquireFullTextConfigWriteLock(locks, dataverseName, configName);
+ }
+
+ @Override
public void createAdapterBegin(IMetadataLockManager lockMgr, LockList locks, DataverseName dataverseName,
String adapterName, DataverseName libraryDataverseName, String libraryName) throws AlgebricksException {
lockMgr.acquireDataverseReadLock(locks, dataverseName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
index 2108a4c..e00d7b6 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/RTreeResourceFactoryProvider.java
@@ -71,15 +71,16 @@
ARecordType recordType, ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory,
Map<String, String> mergePolicyProperties, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories) throws AlgebricksException {
- if (index.getKeyFieldNames().size() != 1) {
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ if (indexDetails.getKeyFieldNames().size() != 1) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD,
- index.getKeyFieldNames().size(), index.getIndexType(), 1);
+ indexDetails.getKeyFieldNames().size(), index.getIndexType(), 1);
}
- IAType spatialType = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), recordType).first;
+ IAType spatialType = Index.getNonNullableOpenFieldType(indexDetails.getKeyFieldTypes().get(0),
+ indexDetails.getKeyFieldNames().get(0), recordType).first;
if (spatialType == null) {
throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
- StringUtils.join(index.getKeyFieldNames().get(0), '.'));
+ StringUtils.join(indexDetails.getKeyFieldNames().get(0), '.'));
}
List<List<String>> primaryKeyFields = dataset.getPrimaryKeys();
int numPrimaryKeys = primaryKeyFields.size();
@@ -189,7 +190,8 @@
private static ITypeTraits[] getTypeTraits(MetadataProvider metadataProvider, Dataset dataset, Index index,
ARecordType recordType, ARecordType metaType) throws AlgebricksException {
ITypeTraitProvider ttProvider = metadataProvider.getStorageComponentProvider().getTypeTraitProvider();
- List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
int numSecondaryKeys = secondaryKeyFields.size();
int numPrimaryKeys = dataset.getPrimaryKeys().size();
ITypeTraits[] primaryTypeTraits = dataset.getPrimaryTypeTraits(metadataProvider, recordType, metaType);
@@ -198,14 +200,14 @@
+ "There can be only one field as a key for the R-tree index.");
}
ARecordType sourceType;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- secondaryKeyFields.get(0), sourceType);
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
IAType spatialType = spatialTypePair.first;
if (spatialType == null) {
throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -227,21 +229,22 @@
ARecordType recordType, ARecordType metaType) throws AlgebricksException {
IBinaryComparatorFactoryProvider cmpFactoryProvider =
metadataProvider.getStorageComponentProvider().getComparatorFactoryProvider();
- List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
int numSecondaryKeys = secondaryKeyFields.size();
if (numSecondaryKeys != 1) {
throw new AsterixException("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.");
}
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
ARecordType sourceType;
if (keySourceIndicators == null || keySourceIndicators.get(0) == 0) {
sourceType = recordType;
} else {
sourceType = metaType;
}
- Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- secondaryKeyFields.get(0), sourceType);
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), sourceType);
IAType spatialType = spatialTypePair.first;
if (spatialType == null) {
throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
new file mode 100644
index 0000000..fdb21a2
--- /dev/null
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryArrayIndexBTreeOperationsHelper.java
@@ -0,0 +1,757 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.metadata.utils;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.Stack;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.utils.StorageConstants;
+import org.apache.asterix.external.indexing.IndexingConstants;
+import org.apache.asterix.formats.nontagged.TypeTraitProvider;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.Index;
+import org.apache.asterix.metadata.entities.InternalDatasetDetails;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.utils.RuntimeUtils;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.algebricks.runtime.operators.aggreg.SimpleAlgebricksAccumulatingAggregatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
+import org.apache.hyracks.algebricks.runtime.operators.std.UnnestRuntimeFactory;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.group.AbstractAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+
+public class SecondaryArrayIndexBTreeOperationsHelper extends SecondaryTreeIndexOperationsHelper {
+ private final int numAtomicSecondaryKeys, numArraySecondaryKeys, numTotalSecondaryKeys;
+ private final Index.ArrayIndexDetails arrayIndexDetails;
+ private final EvalFactoryAndRecDescStackBuilder evalFactoryAndRecDescStackBuilder =
+ new EvalFactoryAndRecDescStackBuilder();
+
+ // TODO (GLENN): Phase these out and use the UNNEST / PROJECT scheme instead.
+ private final List<List<String>> flattenedFieldNames;
+ private final List<IAType> flattenedKeyTypes;
+ private final List<List<Integer>> depthIndicators;
+
+ protected SecondaryArrayIndexBTreeOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
+ SourceLocation sourceLoc) throws AlgebricksException {
+ super(dataset, index, metadataProvider, sourceLoc);
+ arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+
+ flattenedFieldNames = new ArrayList<>();
+ flattenedKeyTypes = new ArrayList<>();
+ depthIndicators = new ArrayList<>();
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ if (e.getUnnestList().isEmpty()) {
+ flattenedFieldNames.add(e.getProjectList().get(0));
+ flattenedKeyTypes.add(e.getTypeList().get(0));
+ depthIndicators
+ .add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(0)));
+ } else {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ List<String> project = e.getProjectList().get(i);
+ flattenedFieldNames.add(ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project));
+ depthIndicators.add(ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project));
+ flattenedKeyTypes.add(e.getTypeList().get(i));
+ }
+ }
+ }
+
+ int totalSecondaryKeyCount = 0;
+ int atomicSecondaryKeyCount = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ if (e.getUnnestList().isEmpty()) {
+ atomicSecondaryKeyCount++;
+ totalSecondaryKeyCount++;
+ } else {
+ totalSecondaryKeyCount += e.getProjectList().size();
+ }
+ }
+
+ numTotalSecondaryKeys = totalSecondaryKeyCount;
+ numAtomicSecondaryKeys = atomicSecondaryKeyCount;
+ numArraySecondaryKeys = numTotalSecondaryKeys - numAtomicSecondaryKeys;
+ }
+
+ private int findPosOfArrayIndex() throws AsterixException {
+ for (int i = 0; i < arrayIndexDetails.getElementList().size(); i++) {
+ if (!arrayIndexDetails.getElementList().get(i).getUnnestList().isEmpty()) {
+ return i;
+ }
+ }
+ throw new AsterixException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, "No array index found.");
+ }
+
+ @Override
+ protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
+ Index.ArrayIndexDetails arrayIndexDetails = (Index.ArrayIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = this.getNumSecondaryKeys();
+ secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
+ secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+ secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
+ ISerializerDeserializer[] secondaryRecFields =
+ new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys + numFilterFields];
+ ISerializerDeserializer[] enforcedRecFields =
+ new ISerializerDeserializer[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
+ ITypeTraits[] enforcedTypeTraits =
+ new ITypeTraits[1 + numPrimaryKeys + (dataset.hasMetaPart() ? 1 : 0) + numFilterFields];
+ secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+ ITypeTraitProvider typeTraitProvider = metadataProvider.getDataFormat().getTypeTraitProvider();
+ IBinaryComparatorFactoryProvider comparatorFactoryProvider =
+ metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
+ // Record column is 0 for external datasets, numPrimaryKeys for internal ones
+ int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
+ boolean isOverridingKeyFieldTypes = arrayIndexDetails.isOverridingKeyFieldTypes();
+ int flattenedListPos = 0;
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ for (int i = 0; i < e.getProjectList().size(); i++) {
+ ARecordType sourceType = (e.getSourceIndicator() == 0) ? itemType : metaType;
+ addSKEvalFactories(isOverridingKeyFieldTypes ? enforcedItemType : sourceType, flattenedListPos, false);
+ Pair<IAType, Boolean> keyTypePair = ArrayIndexUtil.getNonNullableOpenFieldType(e.getTypeList().get(i),
+ ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), e.getProjectList().get(i)),
+ sourceType,
+ ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), e.getProjectList().get(i)));
+ IAType keyType = keyTypePair.first;
+ anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
+ ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
+ secondaryRecFields[flattenedListPos] = keySerde;
+ secondaryComparatorFactories[flattenedListPos] =
+ comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+ secondaryTypeTraits[flattenedListPos] = typeTraitProvider.getTypeTrait(keyType);
+ secondaryBloomFilterKeyFields[flattenedListPos] = flattenedListPos;
+
+ flattenedListPos++;
+ }
+ }
+ if (dataset.getDatasetType() == DatasetType.INTERNAL) {
+ // Add serializers and comparators for primary index fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+ enforcedRecFields[i] = primaryRecDesc.getFields()[i];
+ secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+ enforcedTypeTraits[i] = primaryRecDesc.getTypeTraits()[i];
+ secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+ }
+ } else {
+ // Add serializers and comparators for RID fields.
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ secondaryRecFields[numSecondaryKeys + i] = IndexingConstants.getSerializerDeserializer(i);
+ enforcedRecFields[i] = IndexingConstants.getSerializerDeserializer(i);
+ secondaryTypeTraits[numSecondaryKeys + i] = IndexingConstants.getTypeTraits(i);
+ enforcedTypeTraits[i] = IndexingConstants.getTypeTraits(i);
+ secondaryComparatorFactories[numSecondaryKeys + i] = IndexingConstants.getComparatorFactory(i);
+ }
+ }
+ enforcedRecFields[numPrimaryKeys] = serdeProvider.getSerializerDeserializer(itemType);
+ enforcedTypeTraits[numPrimaryKeys] = typeTraitProvider.getTypeTrait(itemType);
+ if (dataset.hasMetaPart()) {
+ enforcedRecFields[numPrimaryKeys + 1] = serdeProvider.getSerializerDeserializer(metaType);
+ enforcedTypeTraits[numPrimaryKeys + 1] = typeTraitProvider.getTypeTrait(metaType);
+ }
+
+ if (numFilterFields > 0) {
+ ARecordType filterItemType =
+ ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterSourceIndicator() == 0 ? itemType
+ : metaType;
+ addSKEvalFactories(itemType, numSecondaryKeys, true);
+ Pair<IAType, Boolean> keyTypePair;
+ keyTypePair = Index.getNonNullableKeyFieldType(filterFieldName, filterItemType);
+ IAType type = keyTypePair.first;
+ ISerializerDeserializer serde = serdeProvider.getSerializerDeserializer(type);
+ secondaryRecFields[numPrimaryKeys + numSecondaryKeys] = serde;
+ enforcedRecFields[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] = serde;
+ enforcedTypeTraits[numPrimaryKeys + 1 + (dataset.hasMetaPart() ? 1 : 0)] =
+ typeTraitProvider.getTypeTrait(type);
+ }
+ secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+ enforcedRecDesc = new RecordDescriptor(enforcedRecFields, enforcedTypeTraits);
+ }
+
+ @Override
+ protected int getNumSecondaryKeys() {
+ return arrayIndexDetails.getElementList().stream().map(e -> e.getProjectList().size()).reduce(0, Integer::sum);
+ }
+
+ protected int[] createFieldPermutationForBulkLoadOp(int numSecondaryKeyFields) {
+ int[] fieldPermutation = new int[numSecondaryKeyFields + numPrimaryKeys + numFilterFields];
+ for (int i = 0; i < fieldPermutation.length; i++) {
+ fieldPermutation[i] = i;
+ }
+ return fieldPermutation;
+ }
+
+ protected void addSKEvalFactories(ARecordType recordType, int fieldPos, boolean isFilterField)
+ throws AlgebricksException {
+ if (isFilterField) {
+ addFilterFieldToBuilder(recordType);
+ return;
+ }
+
+ List<Integer> arrayDepthIndicators = depthIndicators.get(fieldPos);
+ List<String> fieldNames = flattenedFieldNames.get(fieldPos);
+ if (arrayDepthIndicators.stream().noneMatch(b -> b > 0)) {
+ addAtomicFieldToBuilder(recordType, fieldPos);
+ } else {
+ EvalFactoryAndRecDescInvoker commandExecutor =
+ new EvalFactoryAndRecDescInvoker(!evalFactoryAndRecDescStackBuilder.isUnnestEvalPopulated());
+ ArrayIndexUtil.walkArrayPath(recordType, fieldNames, arrayDepthIndicators, commandExecutor);
+ }
+ }
+
+ /**
+ * The following job spec is produced: (key provider) -> (PIDX scan) -> (cast)? -> ((unnest) -> (assign))* ->
+ * (select)? -> (sort)? -> (bulk load) -> (sink)
+ */
+ @Override
+ public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
+ JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ throw new UnsupportedOperationException("Array indexes on external datasets not currently supported.");
+ } else {
+ IndexUtil.bindJobEventListener(spec, metadataProvider);
+
+ // Start the job spec. Create a key provider and connect this to a primary index scan.
+ IOperatorDescriptor sourceOp = DatasetUtil.createDummyKeyProviderOp(spec, dataset, metadataProvider);
+ IOperatorDescriptor targetOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+ sourceOp = targetOp;
+ if (arrayIndexDetails.isOverridingKeyFieldTypes() && !enforcedItemType.equals(itemType)) {
+ // If we have an enforced type, insert a "cast" after the primary index scan.
+ targetOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+ }
+
+ // TODO (GLENN): Refactor to use UNNEST + PROJECT scheme.
+ // Perform the unnest work.
+ final Mutable<IOperatorDescriptor> sourceOpRef = new MutableObject<>(sourceOp);
+ final Mutable<IOperatorDescriptor> targetOpRef = new MutableObject<>(targetOp);
+ LoadingJobBuilder jobBuilder = new LoadingJobBuilder(spec, sourceOpRef, targetOpRef);
+ int posOfArrayIndex = findPosOfArrayIndex();
+ ArrayIndexUtil.walkArrayPath(flattenedFieldNames.get(posOfArrayIndex), depthIndicators.get(posOfArrayIndex),
+ jobBuilder);
+ sourceOp = sourceOpRef.getValue();
+
+ if (anySecondaryKeyIsNullable || arrayIndexDetails.isOverridingKeyFieldTypes()) {
+ // If any of the secondary fields are nullable, then we need to filter out the nulls.
+ targetOp = createFilterNullsSelectOp(spec, numTotalSecondaryKeys, secondaryRecDesc);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+ }
+
+ // Sort by secondary keys, then primary keys.
+ IBinaryComparatorFactory[] comparatorFactories = getComparatorFactoriesForOrder();
+ targetOp = createSortOp(spec, comparatorFactories, secondaryRecDesc);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // Only insert unique <SK, PK> pairs into our index,
+ targetOp = createPreSortedDistinctOp(spec, comparatorFactories, secondaryRecDesc);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ sourceOp = targetOp;
+
+ // Apply the bulk loading operator.
+ IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
+ metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
+ targetOp = createTreeIndexBulkLoadOp(spec, createFieldPermutationForBulkLoadOp(numTotalSecondaryKeys),
+ dataflowHelperFactory, StorageConstants.DEFAULT_TREE_FILL_FACTOR);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+
+ // Apply the sink.
+ sourceOp = targetOp;
+ SinkRuntimeFactory sinkRuntimeFactory = new SinkRuntimeFactory();
+ sinkRuntimeFactory.setSourceLocation(sourceLoc);
+ targetOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 0,
+ new IPushRuntimeFactory[] { sinkRuntimeFactory }, new RecordDescriptor[] { secondaryRecDesc });
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
+ spec.addRoot(targetOp);
+ spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+ return spec;
+ }
+ }
+
+ private IBinaryComparatorFactory[] getComparatorFactoriesForOrder() {
+ IBinaryComparatorFactory[] comparatorFactories =
+ new IBinaryComparatorFactory[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+ if (numTotalSecondaryKeys >= 0) {
+ System.arraycopy(secondaryComparatorFactories, 0, comparatorFactories, 0, numTotalSecondaryKeys);
+ }
+ if (numPrimaryKeys >= 0) {
+ System.arraycopy(primaryComparatorFactories, 0, comparatorFactories, numTotalSecondaryKeys, numPrimaryKeys);
+ }
+ if (numFilterFields > 0) {
+ comparatorFactories[numTotalSecondaryKeys + numPrimaryKeys] =
+ secondaryComparatorFactories[numTotalSecondaryKeys];
+ }
+ return comparatorFactories;
+ }
+
+ private IOperatorDescriptor createPreSortedDistinctOp(JobSpecification spec,
+ IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc) {
+ int[] distinctFields = new int[secondaryComparatorFactories.length];
+ for (int i = 0; i < secondaryComparatorFactories.length; i++) {
+ distinctFields[i] = i;
+ }
+
+ IAggregateEvaluatorFactory[] aggFactories = new IAggregateEvaluatorFactory[] {};
+ AbstractAggregatorDescriptorFactory aggregatorFactory =
+ new SimpleAlgebricksAccumulatingAggregatorFactory(aggFactories, distinctFields);
+ aggregatorFactory.setSourceLocation(sourceLoc);
+
+ PreclusteredGroupOperatorDescriptor distinctOp = new PreclusteredGroupOperatorDescriptor(spec, distinctFields,
+ secondaryComparatorFactories, aggregatorFactory, secondaryRecDesc, false, -1);
+ distinctOp.setSourceLocation(sourceLoc);
+
+ return distinctOp;
+ }
+
+ /**
+ * Create an UNNEST operator for use with array indexes, which will perform the unnest and append the new field to
+ * the end of the input tuple. We expect three types of inputs to this operator:
+ * <p>
+ * <ol>
+ * <li>Tuples from a PIDX scan, which are in the format [PKs, record].
+ * <li>Tuples from an UNNEST op, which are in the format [PKs, (filter)?, intermediate-record].
+ * <li>Tuples from an UNNEST op that has already assigned a composite key, which are in the format:
+ * [PKs, (atomic SKs)?, (filter)?, intermediate-record].
+ * </ol>
+ * <p>
+ * In all cases here, the field(s) we want to unnest are located at the end of the input tuple.
+ */
+ private AlgebricksMetaOperatorDescriptor createUnnestOp(JobSpecification spec, int inputWidth,
+ IScalarEvaluatorFactory sef, RecordDescriptor unnestRecDesc) throws AlgebricksException {
+ int[] projectionList = IntStream.range(0, inputWidth + 1).toArray();
+ IUnnestingEvaluatorFactory unnestingEvaluatorFactory =
+ metadataProvider.getFunctionManager().lookupFunction(BuiltinFunctions.SCAN_COLLECTION, sourceLoc)
+ .createUnnestingEvaluatorFactory(new IScalarEvaluatorFactory[] { sef });
+ UnnestRuntimeFactory unnest = new UnnestRuntimeFactory(projectionList.length - 1, unnestingEvaluatorFactory,
+ projectionList, false, null);
+ unnest.setSourceLocation(sourceLoc);
+ AlgebricksMetaOperatorDescriptor algebricksMetaOperatorDescriptor = new AlgebricksMetaOperatorDescriptor(spec,
+ 1, 1, new IPushRuntimeFactory[] { unnest }, new RecordDescriptor[] { unnestRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, algebricksMetaOperatorDescriptor,
+ primaryPartitionConstraint);
+ return algebricksMetaOperatorDescriptor;
+ }
+
+ /**
+ * Create an ASSIGN operator for use in-between UNNEST operators. This means that the projected fields will be in
+ * the order of [PKs, (atomic SKs)?, (filter)?, intermediate record], from the following expected inputs:
+ * <p>
+ * <ol>
+ * <li>Tuples from an PIDX scan -> UNNEST op, which are in the format [PKs, record, intermediate record].
+ * <li>Tuples from an ASSIGN op -> UNNEST op, which are in the format [PKs, (atomic SKs)?, (filter)?, record,
+ * intermediate record].
+ * <p>
+ * </ol>
+ * In addition to removing the record filter for the first case, we must also retrieve the filter field and the
+ * top-level atomic SK if they exist.
+ */
+ private AlgebricksMetaOperatorDescriptor createIntermediateAssignOp(JobSpecification spec, boolean isFirstAssign,
+ int inputWidth, List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc) {
+ int[] outColumns, projectionList;
+ if (isFirstAssign) {
+ projectionList = new int[numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields + 1];
+ outColumns = IntStream.range(inputWidth, (numAtomicSecondaryKeys + numFilterFields == 1) ? (inputWidth + 1)
+ : inputWidth + numAtomicSecondaryKeys + numFilterFields).toArray();
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ projectionList[i] = i;
+ }
+ System.arraycopy(outColumns, 0, projectionList, numPrimaryKeys, numAtomicSecondaryKeys);
+ if (numFilterFields > 0) {
+ projectionList[numPrimaryKeys + numAtomicSecondaryKeys] = outColumns[outColumns.length - 1];
+ }
+ } else {
+ outColumns = new int[] { inputWidth };
+ projectionList = new int[inputWidth - 1];
+ for (int i = 0; i < projectionList.length - 1; i++) {
+ projectionList[i] = i;
+ }
+ }
+ projectionList[projectionList.length - 1] = inputWidth - 1;
+ return createGenericAssignOp(spec, sefs, assignRecDesc, outColumns, projectionList);
+ }
+
+ /**
+ * Create an ASSIGN operator for use after the final UNNEST operator for an array index bulk-loading job. This means
+ * that the projected fields will be in the order of [SKs, PKs, filter], from the following expected inputs:
+ * <p>
+ * <ol>
+ * <li>Tuples from an PIDX scan -> UNNEST op, which are in the format [PKs, record, intermediate record].
+ * <li>Tuples from an ASSIGN op -> UNNEST op, which are in the format [PKs, (atomic SKs)?, (filter)?, record,
+ * intermediate record].
+ * <p>
+ * </ol>
+ * For the first case, we must also retrieve the filter field and the top-level atomic SK if they exist.
+ */
+ private AlgebricksMetaOperatorDescriptor createFinalAssignOp(JobSpecification spec, boolean isFirstAssign,
+ int inputWidth, List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc) {
+ int[] outColumns, projectionList;
+ if (isFirstAssign) {
+ int outColumnsCursor = 0;
+ projectionList = new int[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+ outColumns = IntStream.range(inputWidth, (numTotalSecondaryKeys + numFilterFields == 1) ? (inputWidth + 1)
+ : inputWidth + numTotalSecondaryKeys + numFilterFields).toArray();
+ for (int i = 0; i < numTotalSecondaryKeys; i++) {
+ int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
+ if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) != 0
+ && (depthIndicators.get(i).stream().anyMatch(b -> b > 0))) {
+ projectionList[i] = numPrimaryKeys + 1;
+ } else {
+ projectionList[i] = outColumns[outColumnsCursor++];
+ }
+ }
+ for (int i = numTotalSecondaryKeys; i < numPrimaryKeys + numTotalSecondaryKeys; i++) {
+ projectionList[i] = i - numTotalSecondaryKeys;
+ }
+ if (numFilterFields > 0) {
+ projectionList[projectionList.length - 1] = outColumns[outColumnsCursor];
+ }
+ } else {
+ int atomicSKCursor = 0, arraySKCursor = 0;
+ projectionList = new int[numPrimaryKeys + numTotalSecondaryKeys + numFilterFields];
+ outColumns = IntStream.range(inputWidth, inputWidth + numArraySecondaryKeys).toArray();
+ for (int i = 0; i < numTotalSecondaryKeys; i++) {
+ int sizeOfFieldNamesForI = flattenedFieldNames.get(i).size();
+ if (depthIndicators.get(i).stream().noneMatch(b -> b > 0)) {
+ projectionList[i] = numPrimaryKeys + atomicSKCursor++;
+ } else if (depthIndicators.get(i).get(sizeOfFieldNamesForI - 1) == 0) {
+ projectionList[i] = outColumns[arraySKCursor++];
+ } else {
+ projectionList[i] = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields + 1;
+ }
+ }
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ projectionList[i + numTotalSecondaryKeys] = i;
+ }
+ if (numFilterFields > 0) {
+ projectionList[numPrimaryKeys + numTotalSecondaryKeys] = numPrimaryKeys + numAtomicSecondaryKeys;
+ }
+ }
+ return createGenericAssignOp(spec, sefs, assignRecDesc, outColumns, projectionList);
+ }
+
+ private AlgebricksMetaOperatorDescriptor createGenericAssignOp(JobSpecification spec,
+ List<IScalarEvaluatorFactory> sefs, RecordDescriptor assignRecDesc, int[] outColumns,
+ int[] projectionList) {
+ AssignRuntimeFactory assign =
+ new AssignRuntimeFactory(outColumns, sefs.toArray(new IScalarEvaluatorFactory[0]), projectionList);
+ assign.setSourceLocation(sourceLoc);
+ AlgebricksMetaOperatorDescriptor algebricksMetaOperatorDescriptor = new AlgebricksMetaOperatorDescriptor(spec,
+ 1, 1, new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { assignRecDesc });
+ AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, algebricksMetaOperatorDescriptor,
+ primaryPartitionConstraint);
+ return algebricksMetaOperatorDescriptor;
+ }
+
+ private void addAtomicFieldToBuilder(ARecordType recordType, int indexPos) throws AlgebricksException {
+ IAType workingType = Index.getNonNullableOpenFieldType(flattenedKeyTypes.get(indexPos),
+ flattenedFieldNames.get(indexPos), recordType).first;
+ IScalarEvaluatorFactory sef =
+ metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(metadataProvider.getFunctionManager(),
+ recordType, flattenedFieldNames.get(indexPos), numPrimaryKeys, sourceLoc);
+ evalFactoryAndRecDescStackBuilder.addAtomicSK(sef, workingType);
+ }
+
+ private void addFilterFieldToBuilder(ARecordType recordType) throws AlgebricksException {
+ IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ metadataProvider.getFunctionManager(), recordType, filterFieldName, numPrimaryKeys, sourceLoc);
+ evalFactoryAndRecDescStackBuilder.addFilter(sef,
+ Index.getNonNullableKeyFieldType(filterFieldName, itemType).first);
+ }
+
+ class EvalFactoryAndRecDescInvoker implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+ private final boolean isFirstWalk;
+
+ public EvalFactoryAndRecDescInvoker(boolean isFirstWalk) {
+ this.isFirstWalk = isFirstWalk;
+ }
+
+ @Override
+ public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+ List<String> fieldName, boolean isFirstArrayStep, boolean isFirstUnnestInStep,
+ boolean isLastUnnestInIntermediateStep) throws AlgebricksException {
+ if (!this.isFirstWalk) {
+ // We have already added the appropriate UNNESTs.
+ return;
+ }
+
+ int sourceColumnForNestedArrays = numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields;
+ if (isFirstUnnestInStep) {
+ int sourceColumnForFirstUnnestInAtomicPath =
+ isFirstArrayStep ? numPrimaryKeys : sourceColumnForNestedArrays;
+ IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
+ sourceColumnForFirstUnnestInAtomicPath, sourceLoc);
+ evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
+ } else {
+ IScalarEvaluatorFactory sef = new ColumnAccessEvalFactory(sourceColumnForNestedArrays);
+ evalFactoryAndRecDescStackBuilder.addUnnest(sef, workingType);
+ }
+ }
+
+ @Override
+ public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+ boolean isNonArrayStep, boolean requiresOnlyOneUnnest) throws AlgebricksException {
+ // If the final value is nested inside a record, add this SEF.
+ if (!isNonArrayStep) {
+ return;
+ }
+
+ int sourceColumnForFinalEvaluator = 1 + ((requiresOnlyOneUnnest) ? numPrimaryKeys
+ : (numPrimaryKeys + numAtomicSecondaryKeys + numFilterFields));
+ IScalarEvaluatorFactory sef = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
+ metadataProvider.getFunctionManager(), startingStepRecordType, fieldName,
+ sourceColumnForFinalEvaluator, sourceLoc);
+ evalFactoryAndRecDescStackBuilder.addFinalArraySK(sef);
+ }
+ }
+
+ class LoadingJobBuilder implements ArrayIndexUtil.ActionCounterCommandExecutor {
+ private final Stack<RecordDescriptor> recDescStack = evalFactoryAndRecDescStackBuilder.buildRecDescStack();
+ private final Stack<List<IScalarEvaluatorFactory>> sefStack =
+ evalFactoryAndRecDescStackBuilder.buildEvalFactoryStack();
+
+ private final JobSpecification spec;
+ private final Mutable<IOperatorDescriptor> sourceOpRef;
+ private final Mutable<IOperatorDescriptor> targetOpRef;
+ private RecordDescriptor workingRecDesc = recDescStack.pop(), nextRecDesc;
+
+ LoadingJobBuilder(JobSpecification spec, Mutable<IOperatorDescriptor> sourceOpRef,
+ Mutable<IOperatorDescriptor> targetOpRef) throws AlgebricksException {
+ this.spec = spec;
+ this.sourceOpRef = sourceOpRef;
+ this.targetOpRef = targetOpRef;
+ }
+
+ private void connectAndMoveToNextOp() {
+ spec.connect(new OneToOneConnectorDescriptor(spec), sourceOpRef.getValue(), 0, targetOpRef.getValue(), 0);
+ sourceOpRef.setValue(targetOpRef.getValue());
+ workingRecDesc = nextRecDesc;
+ }
+
+ @Override
+ public void executeActionOnFirstArrayStep() throws AlgebricksException {
+ nextRecDesc = recDescStack.pop();
+ targetOpRef
+ .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+ connectAndMoveToNextOp();
+ }
+
+ @Override
+ public void executeActionOnIntermediateArrayStep(int numberOfActionsAlreadyPerformed)
+ throws AlgebricksException {
+ // The purpose of the ASSIGN added here is twofold: 1) is to remove the unnecessary record/list we
+ // just unnested, and 2) is to extract the appropriate record fields, if we expect a record next.
+ nextRecDesc = recDescStack.pop();
+ targetOpRef.setValue(createIntermediateAssignOp(spec, numberOfActionsAlreadyPerformed < 2,
+ workingRecDesc.getFieldCount(), sefStack.pop(), nextRecDesc));
+ connectAndMoveToNextOp();
+
+ nextRecDesc = recDescStack.pop();
+ targetOpRef
+ .setValue(createUnnestOp(spec, workingRecDesc.getFieldCount(), sefStack.pop().get(0), nextRecDesc));
+ connectAndMoveToNextOp();
+ }
+
+ @Override
+ public void executeActionOnFinalArrayStep(int numberOfActionsAlreadyPerformed) {
+ targetOpRef.setValue(createFinalAssignOp(spec, numberOfActionsAlreadyPerformed < 2,
+ workingRecDesc.getFieldCount(), sefStack.pop(), recDescStack.pop()));
+ connectAndMoveToNextOp();
+ }
+ }
+
+ class EvalFactoryAndRecDescStackBuilder {
+ private final Stack<IScalarEvaluatorFactory> unnestEvalFactories = new Stack<>();
+ private final List<IScalarEvaluatorFactory> atomicSKEvalFactories = new ArrayList<>();
+ private final List<IScalarEvaluatorFactory> finalArraySKEvalFactories = new ArrayList<>();
+ private final Queue<IAType> unnestEvalTypes = new LinkedList<>();
+ private final List<IAType> atomicSKEvalTypes = new ArrayList<>();
+ private IScalarEvaluatorFactory filterEvalFactory = null;
+ private IAType filterEvalType = null;
+
+ public void addAtomicSK(IScalarEvaluatorFactory sef, IAType type) {
+ atomicSKEvalFactories.add(sef);
+ atomicSKEvalTypes.add(type);
+ }
+
+ public void addFilter(IScalarEvaluatorFactory sef, IAType type) {
+ filterEvalFactory = sef;
+ filterEvalType = type;
+ }
+
+ public void addFinalArraySK(IScalarEvaluatorFactory sef) {
+ finalArraySKEvalFactories.add(sef);
+ }
+
+ public void addUnnest(IScalarEvaluatorFactory sef, IAType type) {
+ unnestEvalFactories.push(sef);
+ unnestEvalTypes.add(type);
+ }
+
+ public boolean isUnnestEvalPopulated() {
+ return !unnestEvalFactories.isEmpty();
+ }
+
+ /**
+ * Order our scalar evaluator factory stack in the order each UNNEST and ASSIGN op will be performed.
+ * <p>
+ * Visually, our stack looks like:
+ *
+ * <pre>
+ * [ first UNNEST SEF ------------------------------------------------- ]
+ * [ first ASSIGN SEFs -- atomic SKs and filter ---------------------- ]
+ * *[ any intermediate UNNEST SEFs --column accessors / record accessors ]
+ * [ final ASSIGN SEFs -- array SKs (record accessors) ---------------- ]
+ * </pre>
+ */
+ public Stack<List<IScalarEvaluatorFactory>> buildEvalFactoryStack() {
+ Stack<List<IScalarEvaluatorFactory>> resultant = new Stack<>();
+ resultant.push(finalArraySKEvalFactories);
+ int initialUnnestEvalFactorySize = unnestEvalFactories.size();
+ for (int i = 0; i < initialUnnestEvalFactorySize - 1; i++) {
+ if (i != 0) {
+ resultant.push(new ArrayList<>());
+ }
+ resultant.push(Collections.singletonList(unnestEvalFactories.pop()));
+ if (i == initialUnnestEvalFactorySize - 2) {
+ resultant.push(new ArrayList<>());
+ }
+ }
+ resultant.peek().addAll(atomicSKEvalFactories);
+ if (filterEvalFactory != null) {
+ resultant.peek().add(filterEvalFactory);
+ }
+ resultant.push(Collections.singletonList(unnestEvalFactories.pop()));
+ return resultant;
+ }
+
+ /**
+ * Order our record descriptor stack in the same order as our SEF stack.
+ * <p>
+ * Visually, our stack looks like:
+ *
+ * <pre>
+ * [ primary record descriptor --------------------------------------- ]
+ * [ primary record descriptor w/ first UNNESTed field at the end ---- ]
+ * [ record descriptor w/ atomic fields, w/o record, w/ UNNESTed field ]
+ * *[ same record descriptor as above, w/ new unnested field ---------- ]
+ * *[ same record descriptor as above, w/o record field --------------- ]
+ * [ secondary record descriptor ------------------------------------- ]
+ * </pre>
+ */
+ public Stack<RecordDescriptor> buildRecDescStack() throws AlgebricksException {
+ int initialUnnestEvalTypesSize = unnestEvalTypes.size();
+ Deque<RecordDescriptor> resultantAsDeque = new ArrayDeque<>();
+ resultantAsDeque.addFirst(primaryRecDesc);
+ resultantAsDeque.addFirst(createUnnestRecDesc(primaryRecDesc, unnestEvalTypes.remove()));
+ for (int i = 0; i < initialUnnestEvalTypesSize - 1; i++) {
+ resultantAsDeque.addFirst(createAssignRecDesc(resultantAsDeque.getFirst(), i == 0));
+ resultantAsDeque.addFirst(createUnnestRecDesc(resultantAsDeque.getFirst(), unnestEvalTypes.remove()));
+ }
+ resultantAsDeque.addFirst(secondaryRecDesc);
+ Stack<RecordDescriptor> resultant = new Stack<>();
+ resultant.addAll(resultantAsDeque);
+ return resultant;
+ }
+
+ private RecordDescriptor createUnnestRecDesc(RecordDescriptor priorRecDesc, IAType type)
+ throws AlgebricksException {
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+ ISerializerDeserializer[] unnestFields = Stream
+ .concat(Stream.of(priorRecDesc.getFields()),
+ Stream.of(serdeProvider.getSerializerDeserializer(type)))
+ .toArray(ISerializerDeserializer[]::new);
+ ITypeTraits[] unnestTypes = Stream.concat(Stream.of(priorRecDesc.getTypeTraits()),
+ Stream.of(TypeTraitProvider.INSTANCE.getTypeTrait(type))).toArray(ITypeTraits[]::new);
+ return new RecordDescriptor(unnestFields, unnestTypes);
+ }
+
+ private RecordDescriptor createAssignRecDesc(RecordDescriptor priorRecDesc, boolean isFirstAssign)
+ throws AlgebricksException {
+ ArrayList<ISerializerDeserializer> assignFields = new ArrayList<>();
+ ArrayList<ITypeTraits> assignTypes = new ArrayList<>();
+ if (isFirstAssign) {
+ ISerializerDeserializerProvider serdeProvider = metadataProvider.getDataFormat().getSerdeProvider();
+ for (int i = 0; i < numPrimaryKeys; i++) {
+ assignFields.add(priorRecDesc.getFields()[i]);
+ assignTypes.add(priorRecDesc.getTypeTraits()[i]);
+ }
+ for (IAType s : atomicSKEvalTypes) {
+ assignFields.add(serdeProvider.getSerializerDeserializer(s));
+ assignTypes.add(TypeTraitProvider.INSTANCE.getTypeTrait(s));
+ }
+ if (filterEvalType != null) {
+ assignFields.add(serdeProvider.getSerializerDeserializer(filterEvalType));
+ assignTypes.add(TypeTraitProvider.INSTANCE.getTypeTrait(filterEvalType));
+ }
+ assignFields.add(priorRecDesc.getFields()[priorRecDesc.getFieldCount() - 1]);
+ assignTypes.add(priorRecDesc.getTypeTraits()[priorRecDesc.getFieldCount() - 1]);
+ } else {
+ assignFields = new ArrayList<>(Arrays.asList(priorRecDesc.getFields()));
+ assignTypes = new ArrayList<>(Arrays.asList(priorRecDesc.getTypeTraits()));
+ assignFields.remove(priorRecDesc.getFieldCount() - 2);
+ assignTypes.remove(priorRecDesc.getFieldCount() - 2);
+ }
+ return new RecordDescriptor(assignFields.toArray(new ISerializerDeserializer[0]),
+ assignTypes.toArray(new ITypeTraits[0]));
+ }
+ }
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
index bc5ac8b..33f5b62 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryBTreeOperationsHelper.java
@@ -65,8 +65,9 @@
@Override
public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
- int[] fieldPermutation = createFieldPermutationForBulkLoadOp(index.getKeyFieldNames().size());
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
+ int[] fieldPermutation = createFieldPermutationForBulkLoadOp(indexDetails.getKeyFieldNames().size());
IIndexDataflowHelperFactory dataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
@@ -86,12 +87,12 @@
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
}
AlgebricksMetaOperatorDescriptor asterixAssignOp =
- createExternalAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+ createExternalAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
// If any of the secondary fields are nullable, then add a select op that filters nulls.
AlgebricksMetaOperatorDescriptor selectOp = null;
if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
- selectOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+ selectOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
}
// Sort by secondary keys.
@@ -144,20 +145,20 @@
sourceOp = targetOp;
}
// primary index OR cast assign ----> assign op
- targetOp = createAssignOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+ targetOp = createAssignOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
if (anySecondaryKeyIsNullable || isOverridingKeyFieldTypes) {
// if any of the secondary fields are nullable, then add a select op that filters nulls.
// assign op ----> select op
- targetOp = createFilterNullsSelectOp(spec, index.getKeyFieldNames().size(), secondaryRecDesc);
+ targetOp = createFilterNullsSelectOp(spec, indexDetails.getKeyFieldNames().size(), secondaryRecDesc);
spec.connect(new OneToOneConnectorDescriptor(spec), sourceOp, 0, targetOp, 0);
sourceOp = targetOp;
}
// no need to sort if the index is secondary primary index
- if (!index.getKeyFieldNames().isEmpty()) {
+ if (!indexDetails.getKeyFieldNames().isEmpty()) {
// sort by secondary keys.
// assign op OR select op ----> sort op
targetOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
@@ -186,7 +187,7 @@
@Override
protected int getNumSecondaryKeys() {
- return index.getKeyFieldNames().size();
+ return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
}
/**
@@ -208,7 +209,8 @@
*/
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
@@ -225,11 +227,11 @@
metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
// Record column is 0 for external datasets, numPrimaryKeys for internal ones
int recordColumn = dataset.getDatasetType() == DatasetType.INTERNAL ? numPrimaryKeys : 0;
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
int sourceColumn;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = itemType;
sourceColumn = recordColumn;
@@ -239,9 +241,9 @@
}
secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : sourceType,
- index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
index 40f2610..aca953b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedBTreeOperationsHelper.java
@@ -66,7 +66,8 @@
public JobSpecification buildLoadingJobSpec() throws AlgebricksException {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
assert dataset.getDatasetType() == DatasetType.INTERNAL;
@@ -89,7 +90,7 @@
}
RecordDescriptor taggedSecondaryRecDesc = getTaggedRecordDescriptor(secondaryRecDesc);
AlgebricksMetaOperatorDescriptor asterixAssignOp =
- createAssignOp(spec, index.getKeyFieldNames().size(), taggedSecondaryRecDesc);
+ createAssignOp(spec, indexDetails.getKeyFieldNames().size(), taggedSecondaryRecDesc);
// Generate compensate tuples for upsert
IOperatorDescriptor processorOp =
@@ -122,12 +123,13 @@
@Override
protected int getNumSecondaryKeys() {
- return index.getKeyFieldNames().size();
+ return ((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
}
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
secondaryFieldAccessEvalFactories = new IScalarEvaluatorFactory[numSecondaryKeys + numFilterFields];
secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
@@ -144,11 +146,11 @@
metadataProvider.getDataFormat().getBinaryComparatorFactoryProvider();
// Record column is 0 for external datasets, numPrimaryKeys for internal ones
int recordColumn = NUM_TAG_FIELDS + numPrimaryKeys;
- boolean isOverridingKeyTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyTypes = indexDetails.isOverridingKeyFieldTypes();
for (int i = 0; i < numSecondaryKeys; i++) {
ARecordType sourceType;
int sourceColumn;
- List<Integer> keySourceIndicators = index.getKeyFieldSourceIndicators();
+ List<Integer> keySourceIndicators = indexDetails.getKeyFieldSourceIndicators();
if (keySourceIndicators == null || keySourceIndicators.get(i) == 0) {
sourceType = itemType;
sourceColumn = recordColumn;
@@ -158,9 +160,9 @@
}
secondaryFieldAccessEvalFactories[i] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyTypes ? enforcedItemType : sourceType,
- index.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(i),
- index.getKeyFieldNames().get(i), sourceType);
+ indexDetails.getKeyFieldNames().get(i), sourceColumn, sourceLoc);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(i), indexDetails.getKeyFieldNames().get(i), sourceType);
IAType keyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
index cf84e56..c111f0e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedInvertedIndexOperationsHelper.java
@@ -55,6 +55,7 @@
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
public class SecondaryCorrelatedInvertedIndexOperationsHelper extends SecondaryCorrelatedTreeIndexOperationsHelper {
@@ -64,6 +65,7 @@
private IBinaryComparatorFactory[] tokenComparatorFactories;
private ITypeTraits[] tokenTypeTraits;
private IBinaryTokenizerFactory tokenizerFactory;
+ private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
// For tokenization, sorting and loading. Represents <token, primary keys>.
private int numTokenKeyPairFields;
private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
@@ -80,9 +82,10 @@
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
IndexType indexType = index.getIndexType();
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
// Sanity checks.
if (numPrimaryKeys > 1) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
@@ -112,9 +115,9 @@
if (numSecondaryKeys > 0) {
secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
- index.getKeyFieldNames().get(0), recordColumn, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), itemType);
+ indexDetails.getKeyFieldNames().get(0), recordColumn, sourceLoc);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
secondaryKeyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -149,7 +152,9 @@
// TODO: We might want to expose the hashing option at the AQL level,
// and add the choice to the index metadata.
tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
- index.getGramLength());
+ indexDetails.getGramLength());
+ fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
+ index.getDataverseName(), indexDetails.getFullTextConfigName());
// Type traits for inverted-list elements. Inverted lists contain
// primary keys.
invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -218,8 +223,9 @@
getTaggedRecordDescriptor(dataset.getPrimaryRecordDescriptor(metadataProvider)));
IOperatorDescriptor sourceOp = primaryScanOp;
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
@@ -268,7 +274,7 @@
private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
int docField = NUM_TAG_FIELDS;
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
int[] keyFields = new int[NUM_TAG_FIELDS + numPrimaryKeys + numFilterFields];
// set tag fields
for (int i = 0; i < NUM_TAG_FIELDS; i++) {
@@ -279,8 +285,8 @@
keyFields[i] = i + numSecondaryKeys;
}
BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
- getTaggedRecordDescriptor(tokenKeyPairRecDesc), tokenizerFactory, docField, keyFields, isPartitioned,
- false, true, MissingWriterFactory.INSTANCE);
+ getTaggedRecordDescriptor(tokenKeyPairRecDesc), tokenizerFactory, fullTextConfigEvaluatorFactory,
+ docField, keyFields, isPartitioned, false, true, MissingWriterFactory.INSTANCE);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
primaryPartitionConstraint);
return tokenizerOp;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
index d10c093..82c67d7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedRTreeOperationsHelper.java
@@ -75,15 +75,16 @@
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
int numSecondaryKeys = secondaryKeyFields.size();
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
if (numSecondaryKeys != 1) {
throw AsterixException.create(ErrorCode.INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED, sourceLoc,
numSecondaryKeys);
}
- Pair<IAType, Boolean> spatialTypePair =
- Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
IAType spatialType = spatialTypePair.first;
anySecondaryKeyIsNullable = spatialTypePair.second;
isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
@@ -182,7 +183,7 @@
RecordDescriptor secondaryRecDescConsideringPointMBR = isPointMBR
? getTaggedRecordDescriptor(secondaryRecDescForPointMBR) : getTaggedRecordDescriptor(secondaryRecDesc);
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
assert dataset.getDatasetType() == DatasetType.INTERNAL;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 3586c21..a6e3087 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -139,7 +139,7 @@
private static Pair<ARecordType, ARecordType> getEnforcedType(Index index, ARecordType aRecordType,
ARecordType metaRecordType) throws AlgebricksException {
- return index.isOverridingKeyFieldTypes()
+ return index.getIndexDetails().isOverridingKeyFieldTypes()
? TypeUtil.createEnforcedType(aRecordType, metaRecordType, Collections.singletonList(index))
: new Pair<>(null, null);
}
@@ -155,6 +155,10 @@
SecondaryIndexOperationsHelper indexOperationsHelper;
switch (index.getIndexType()) {
+ case ARRAY:
+ indexOperationsHelper =
+ new SecondaryArrayIndexBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
+ break;
case BTREE:
indexOperationsHelper = new SecondaryBTreeOperationsHelper(dataset, index, metadataProvider, sourceLoc);
break;
@@ -227,7 +231,7 @@
secondaryFilterFields = new int[numFilterFields];
primaryFilterFields = new int[numFilterFields];
primaryBTreeFields = new int[numPrimaryKeys + 1];
- secondaryBTreeFields = new int[index.getKeyFieldNames().size() + numPrimaryKeys];
+ secondaryBTreeFields = new int[getNumSecondaryKeys() + numPrimaryKeys];
for (int i = 0; i < primaryBTreeFields.length; i++) {
primaryBTreeFields[i] = i;
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
index 85205b3..2d5a4f8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryInvertedIndexOperationsHelper.java
@@ -58,6 +58,7 @@
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
public class SecondaryInvertedIndexOperationsHelper extends SecondaryTreeIndexOperationsHelper {
@@ -67,6 +68,7 @@
private IBinaryComparatorFactory[] tokenComparatorFactories;
private ITypeTraits[] tokenTypeTraits;
private IBinaryTokenizerFactory tokenizerFactory;
+ private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
// For tokenization, sorting and loading. Represents <token, primary keys>.
private int numTokenKeyPairFields;
private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
@@ -79,13 +81,16 @@
protected SecondaryInvertedIndexOperationsHelper(Dataset dataset, Index index, MetadataProvider metadataProvider,
SourceLocation sourceLoc) throws AlgebricksException {
super(dataset, index, metadataProvider, sourceLoc);
+ this.fullTextConfigEvaluatorFactory = FullTextUtil.fetchFilterAndCreateConfigEvaluator(metadataProvider,
+ index.getDataverseName(), ((Index.TextIndexDetails) index.getIndexDetails()).getFullTextConfigName());
}
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- int numSecondaryKeys = index.getKeyFieldNames().size();
IndexType indexType = index.getIndexType();
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ Index.TextIndexDetails indexDetails = (Index.TextIndexDetails) index.getIndexDetails();
+ int numSecondaryKeys = indexDetails.getKeyFieldNames().size();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
// Sanity checks.
if (numPrimaryKeys > 1) {
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX,
@@ -114,9 +119,9 @@
if (numSecondaryKeys > 0) {
secondaryFieldAccessEvalFactories[0] = metadataProvider.getDataFormat().getFieldAccessEvaluatorFactory(
metadataProvider.getFunctionManager(), isOverridingKeyFieldTypes ? enforcedItemType : itemType,
- index.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
- Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0),
- index.getKeyFieldNames().get(0), itemType);
+ indexDetails.getKeyFieldNames().get(0), numPrimaryKeys, sourceLoc);
+ Pair<IAType, Boolean> keyTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), indexDetails.getKeyFieldNames().get(0), itemType);
secondaryKeyType = keyTypePair.first;
anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
@@ -151,7 +156,7 @@
// TODO: We might want to expose the hashing option at the AQL level,
// and add the choice to the index metadata.
tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(), indexType,
- index.getGramLength());
+ ((Index.TextIndexDetails) indexDetails).getGramLength());
// Type traits for inverted-list elements. Inverted lists contain
// primary keys.
invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
@@ -219,8 +224,8 @@
IOperatorDescriptor primaryScanOp = DatasetUtil.createPrimaryIndexScanOp(spec, metadataProvider, dataset);
IOperatorDescriptor sourceOp = primaryScanOp;
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
+ int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
if (isOverridingKeyFieldTypes && !enforcedItemType.equals(itemType)) {
sourceOp = createCastOp(spec, dataset.getDatasetType(), index.isEnforced());
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, sourceOp, 0);
@@ -270,14 +275,14 @@
private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) {
int docField = 0;
- int numSecondaryKeys = index.getKeyFieldNames().size();
+ int numSecondaryKeys = ((Index.TextIndexDetails) index.getIndexDetails()).getKeyFieldNames().size();
int[] primaryKeyFields = new int[numPrimaryKeys + numFilterFields];
for (int i = 0; i < primaryKeyFields.length; i++) {
primaryKeyFields[i] = numSecondaryKeys + i;
}
- BinaryTokenizerOperatorDescriptor tokenizerOp =
- new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc, tokenizerFactory, docField,
- primaryKeyFields, isPartitioned, false, false, MissingWriterFactory.INSTANCE);
+ BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec, tokenKeyPairRecDesc,
+ tokenizerFactory, fullTextConfigEvaluatorFactory, docField, primaryKeyFields, isPartitioned, false,
+ false, MissingWriterFactory.INSTANCE);
tokenizerOp.setSourceLocation(sourceLoc);
AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
primaryPartitionConstraint);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
index 1d60772..dbbd723 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryRTreeOperationsHelper.java
@@ -81,15 +81,16 @@
@Override
protected void setSecondaryRecDescAndComparators() throws AlgebricksException {
- List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
+ Index.ValueIndexDetails indexDetails = (Index.ValueIndexDetails) index.getIndexDetails();
+ List<List<String>> secondaryKeyFields = indexDetails.getKeyFieldNames();
int numSecondaryKeys = secondaryKeyFields.size();
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = indexDetails.isOverridingKeyFieldTypes();
if (numSecondaryKeys != 1) {
throw new AsterixException("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> spatialTypePair =
- Index.getNonNullableOpenFieldType(index.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
+ Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableOpenFieldType(
+ indexDetails.getKeyFieldTypes().get(0), secondaryKeyFields.get(0), itemType);
IAType spatialType = spatialTypePair.first;
anySecondaryKeyIsNullable = spatialTypePair.second;
if (spatialType == null) {
@@ -198,7 +199,7 @@
isPointMBR ? numNestedSecondaryKeyFields / 2 : numNestedSecondaryKeyFields;
RecordDescriptor secondaryRecDescConsideringPointMBR =
isPointMBR ? secondaryRecDescForPointMBR : secondaryRecDesc;
- boolean isOverridingKeyFieldTypes = index.isOverridingKeyFieldTypes();
+ boolean isOverridingKeyFieldTypes = index.getIndexDetails().isOverridingKeyFieldTypes();
IIndexDataflowHelperFactory indexDataflowHelperFactory = new IndexDataflowHelperFactory(
metadataProvider.getStorageComponentProvider().getStorageManager(), secondaryFileSplitProvider);
if (dataset.getDatasetType() == DatasetType.INTERNAL) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index 65a800e..dd303fc 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -27,12 +27,14 @@
import java.util.Map;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.metadata.DataverseName;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.Function;
import org.apache.asterix.metadata.entities.Index;
import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
@@ -42,6 +44,7 @@
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
/**
* Provider utility methods for data types
@@ -57,6 +60,195 @@
private TypeUtil() {
}
+ private static class EnforcedTypeBuilder {
+ private final Deque<Triple<IAType, String, Integer>> typeStack = new ArrayDeque<>();
+ private List<Integer> keyDepthIndicators;
+ private List<String> keyFieldNames;
+ private ARecordType baseRecordType;
+ private IAType keyFieldType;
+
+ // Output from nested-type-stack construction.
+ private String bridgeNameFoundFromOpenTypeBuild;
+ private IAType endOfOpenTypeBuild;
+ private int indexOfOpenPart;
+
+ public void reset(ARecordType baseRecordType, List<String> keyFieldNames, List<Integer> keyDepthIndicators,
+ IAType keyFieldType) {
+ this.baseRecordType = baseRecordType;
+ this.keyFieldNames = keyFieldNames;
+ this.keyDepthIndicators = keyDepthIndicators;
+ this.keyFieldType = keyFieldType;
+ }
+
+ public ARecordType build() throws AlgebricksException {
+ boolean isOpen = constructNestedTypeStack();
+ IAType newTypeToAdd = (isOpen) ? buildNewForOpenType() : buildNewForFullyClosedType();
+ return buildRestOfRecord(newTypeToAdd);
+ }
+
+ private boolean constructNestedTypeStack() throws AlgebricksException {
+ IAType typeIntermediate = baseRecordType;
+ List<String> subFieldName = new ArrayList<>();
+ for (int i = 0; i < keyFieldNames.size() - 1; i++) {
+ typeStack.push(new Triple<>(typeIntermediate, keyFieldNames.get(i),
+ (i == 0) ? 0 : keyDepthIndicators.get(i - 1)));
+ bridgeNameFoundFromOpenTypeBuild = typeIntermediate.getTypeName();
+
+ if (i == 0 || keyDepthIndicators.get(i - 1) == 0) {
+ subFieldName.add(keyFieldNames.get(i));
+ } else {
+ // We have a multi-valued intermediate. Traverse the array first, then add our field name.
+ for (int j = 0; j < keyDepthIndicators.get(i - 1); j++) {
+ typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
+ if (typeIntermediate == null) {
+ String fName = String.join(".", subFieldName);
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "Wrong level of array nesting for field: " + fName);
+ }
+ }
+ subFieldName.add(keyFieldNames.get(i));
+ }
+
+ // Attempt to resolve the type of our working subfield.
+ typeIntermediate = TypeComputeUtils.getActualType(typeIntermediate);
+ typeIntermediate =
+ ((ARecordType) typeIntermediate).getSubFieldType(subFieldName.subList(i, subFieldName.size()));
+
+ if (typeIntermediate == null) {
+ endOfOpenTypeBuild = null;
+ indexOfOpenPart = i;
+ return true;
+ }
+ ATypeTag tt = TypeComputeUtils.getActualType(typeIntermediate).getTypeTag();
+ if (tt != ATypeTag.OBJECT && tt != ATypeTag.ARRAY && tt != ATypeTag.MULTISET) {
+ String fName = String.join(".", subFieldName);
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR,
+ "Field accessor is not defined for \"" + fName + "\" of type " + tt);
+ }
+ }
+
+ endOfOpenTypeBuild = typeIntermediate;
+ indexOfOpenPart = keyFieldNames.size() - 1;
+ return false;
+ }
+
+ private IAType buildNewForOpenType() {
+ int depthOfOpenType = keyDepthIndicators.subList(indexOfOpenPart + 1, keyDepthIndicators.size()).stream()
+ .filter(i -> i != 0).findFirst().orElse(0);
+ IAType resultant = nestArrayType(keyFieldType, depthOfOpenType);
+
+ // Build the type (list or record) that holds the type (list or record) above.
+ resultant = nestArrayType(
+ new ARecordType(keyFieldNames.get(keyFieldNames.size() - 2),
+ new String[] { keyFieldNames.get(keyFieldNames.size() - 1) },
+ new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
+ keyDepthIndicators.get(indexOfOpenPart));
+
+ // Create open part of the nested field.
+ for (int i = keyFieldNames.size() - 3; i > (indexOfOpenPart - 1); i--) {
+ resultant = nestArrayType(
+ new ARecordType(keyFieldNames.get(i), new String[] { keyFieldNames.get(i + 1) },
+ new IAType[] { AUnionType.createUnknownableType(resultant) }, true),
+ keyDepthIndicators.get(i));
+ }
+
+ // Now update the parent to include this optional field, accounting for intermediate arrays.
+ Triple<IAType, String, Integer> gapTriple = this.typeStack.pop();
+ ARecordType parentRecord =
+ (ARecordType) unnestArrayType(TypeComputeUtils.getActualType(gapTriple.first), gapTriple.third);
+ IAType[] parentFieldTypes = ArrayUtils.addAll(parentRecord.getFieldTypes().clone(),
+ AUnionType.createUnknownableType(resultant));
+ resultant = new ARecordType(bridgeNameFoundFromOpenTypeBuild,
+ ArrayUtils.addAll(parentRecord.getFieldNames(), resultant.getTypeName()), parentFieldTypes, true);
+ resultant = keepUnknown(gapTriple.first, nestArrayType(resultant, gapTriple.third));
+
+ return resultant;
+ }
+
+ private IAType buildNewForFullyClosedType() throws AsterixException {
+ // The schema is closed all the way to the field itself.
+ IAType typeIntermediate = TypeComputeUtils.getActualType(endOfOpenTypeBuild);
+ int depthOfOpenType = (indexOfOpenPart == 0) ? 0 : keyDepthIndicators.get(indexOfOpenPart - 1);
+ int depthOfKeyType = keyDepthIndicators.get(indexOfOpenPart);
+ ARecordType lastNestedRecord = (ARecordType) unnestArrayType(typeIntermediate, depthOfOpenType);
+ Map<String, IAType> recordNameTypesMap = createRecordNameTypeMap(lastNestedRecord);
+
+ // If an enforced field already exists, verify that the type is correct.
+ IAType enforcedFieldType = recordNameTypesMap.get(keyFieldNames.get(keyFieldNames.size() - 1));
+ if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
+ && ((AUnionType) enforcedFieldType).isUnknownableType()) {
+ enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
+ }
+ if (enforcedFieldType != null
+ && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(), this.keyFieldType.getTypeTag())) {
+ throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Cannot enforce field \""
+ + String.join(".", this.keyFieldNames) + "\" to have type " + this.keyFieldType);
+ }
+ if (enforcedFieldType == null) {
+ recordNameTypesMap.put(keyFieldNames.get(keyFieldNames.size() - 1),
+ AUnionType.createUnknownableType(nestArrayType(keyFieldType, depthOfKeyType)));
+ }
+
+ // Build the nested record, and account for the wrapping array.
+ IAType resultant = nestArrayType(
+ new ARecordType(lastNestedRecord.getTypeName(), recordNameTypesMap.keySet().toArray(new String[0]),
+ recordNameTypesMap.values().toArray(new IAType[0]), lastNestedRecord.isOpen()),
+ depthOfOpenType);
+ return keepUnknown(endOfOpenTypeBuild, resultant);
+ }
+
+ private ARecordType buildRestOfRecord(IAType newTypeToAdd) {
+ IAType resultant = TypeComputeUtils.getActualType(newTypeToAdd);
+ while (!typeStack.isEmpty()) {
+ Triple<IAType, String, Integer> typeFromStack = typeStack.pop();
+ IAType typeIntermediate = unnestArrayType(typeFromStack.first, typeFromStack.third);
+ ARecordType recordType = (ARecordType) typeIntermediate;
+ IAType[] fieldTypes = recordType.getFieldTypes().clone();
+ fieldTypes[recordType.getFieldIndex(typeFromStack.second)] = resultant;
+
+ typeIntermediate = nestArrayType(new ARecordType(recordType.getTypeName() + "_enforced",
+ recordType.getFieldNames(), fieldTypes, recordType.isOpen()), typeFromStack.third);
+ resultant = keepUnknown(typeFromStack.first, typeIntermediate);
+ }
+ return (ARecordType) resultant;
+ }
+
+ private static Map<String, IAType> createRecordNameTypeMap(ARecordType recordType) {
+ LinkedHashMap<String, IAType> recordNameTypesMap = new LinkedHashMap<>();
+ for (int j = 0; j < recordType.getFieldNames().length; j++) {
+ recordNameTypesMap.put(recordType.getFieldNames()[j], recordType.getFieldTypes()[j]);
+ }
+ return recordNameTypesMap;
+ }
+
+ private static IAType keepUnknown(IAType originalRecordType, IAType updatedRecordType) {
+ if (originalRecordType.getTypeTag() == ATypeTag.UNION) {
+ return AUnionType.createUnknownableType(updatedRecordType, updatedRecordType.getTypeName());
+ }
+ return updatedRecordType;
+ }
+
+ private static IAType nestArrayType(IAType originalType, int depthOfArrays) {
+ IAType resultant = originalType;
+ for (int i = 0; i < depthOfArrays; i++) {
+ resultant =
+ new AOrderedListType(resultant, (i == depthOfArrays - 1) ? originalType.getTypeName() : null);
+ }
+ return resultant;
+ }
+
+ private static IAType unnestArrayType(IAType originalType, int depthOfArrays) {
+ IAType resultant = originalType;
+ for (int i = 0; i < depthOfArrays; i++) {
+ resultant = TypeComputeUtils.extractListItemType(resultant);
+ if (resultant != null) {
+ resultant = TypeComputeUtils.getActualType(resultant);
+ }
+ }
+ return resultant;
+ }
+ }
+
/**
* Merges typed index fields with specified recordType, allowing indexed fields to be optional.
* I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes
@@ -66,109 +258,97 @@
*/
public static Pair<ARecordType, ARecordType> createEnforcedType(ARecordType recordType, ARecordType metaType,
List<Index> indexes) throws AlgebricksException {
- IAType enforcedRecordType = recordType;
- ARecordType enforcedMetaType = metaType;
- List<String> subFieldName;
+ EnforcedTypeBuilder enforcedTypeBuilder = new EnforcedTypeBuilder();
+ ARecordType enforcedRecordType = recordType;
for (Index index : indexes) {
- if (!index.isSecondaryIndex() || !index.isOverridingKeyFieldTypes()) {
+ if (!index.isSecondaryIndex() || !index.getIndexDetails().isOverridingKeyFieldTypes()) {
continue;
}
- if (index.hasMetaFields()) {
- throw new AlgebricksException("Indexing an open field is only supported on the record part");
- }
- for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
- // keeps track of a record type and a field name in that record type
- Deque<Pair<IAType, String>> nestedTypeStack = new ArrayDeque<>();
- List<String> splits = index.getKeyFieldNames().get(i);
- IAType nestedFieldType = enforcedRecordType;
- boolean openRecords = false;
- String bridgeName = nestedFieldType.getTypeName();
- int j;
- // enforcedRecordType must always be/stay as ARecordType
- validateRecord(enforcedRecordType);
- // build the stack for the enforced type, stack of a mixture of ARecord and AUnion(ARecord) types
- // try to build up to the last record field, e.g. for a.b.c.d.e, build up to and including "d"
- for (j = 1; j < splits.size(); j++) {
- nestedTypeStack.push(new Pair<>(nestedFieldType, splits.get(j - 1)));
- bridgeName = nestedFieldType.getTypeName();
- subFieldName = splits.subList(0, j);
- nestedFieldType = ((ARecordType) enforcedRecordType).getSubFieldType(subFieldName);
- if (nestedFieldType == null) {
- openRecords = true;
- break;
- }
- // nestedFieldType (i.e. nested record field) must be either ARecordType or AUnion(ARecordType)
- validateNestedRecord(nestedFieldType, subFieldName);
- }
- if (openRecords) {
- // create the smallest record
- enforcedRecordType = new ARecordType(splits.get(splits.size() - 2),
- new String[] { splits.get(splits.size() - 1) },
- new IAType[] { AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)) }, true);
- // create the open part of the nested field
- for (int k = splits.size() - 3; k > (j - 2); k--) {
- enforcedRecordType = new ARecordType(splits.get(k), new String[] { splits.get(k + 1) },
- new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) }, true);
- }
- // bridge the gap. Update the parent type to include the new optional field, e.g. c.d.e
- Pair<IAType, String> gapPair = nestedTypeStack.pop();
- ARecordType parent = (ARecordType) TypeComputeUtils.getActualType(gapPair.first);
-
- // parent type must be "open" to allow inclusion of the non-declared field
- IAType[] parentFieldTypes = ArrayUtils.addAll(parent.getFieldTypes().clone(),
- new IAType[] { AUnionType.createUnknownableType(enforcedRecordType) });
- enforcedRecordType = new ARecordType(bridgeName,
- ArrayUtils.addAll(parent.getFieldNames(), enforcedRecordType.getTypeName()),
- parentFieldTypes, true);
- // make nullable/missable if the original parent was nullable/missable
- enforcedRecordType = keepUnknown(gapPair.first, (ARecordType) enforcedRecordType);
- } else {
- // schema is closed all the way to the field. Enforced fields are either null or strongly typed
- // e.g. nestedFieldType = a.b.c.d
- ARecordType lastNestedRecord = (ARecordType) TypeComputeUtils.getActualType(nestedFieldType);
- Map<String, IAType> recordNameTypesMap = TypeUtil.createRecordNameTypeMap(lastNestedRecord);
- // if a an enforced field already exists and the type is correct
- IAType enforcedFieldType = recordNameTypesMap.get(splits.get(splits.size() - 1));
- if (enforcedFieldType != null && enforcedFieldType.getTypeTag() == ATypeTag.UNION
- && ((AUnionType) enforcedFieldType).isUnknownableType()) {
- enforcedFieldType = ((AUnionType) enforcedFieldType).getActualType();
- }
- if (enforcedFieldType != null && !ATypeHierarchy.canPromote(enforcedFieldType.getTypeTag(),
- index.getKeyFieldTypes().get(i).getTypeTag())) {
- throw new AsterixException(ErrorCode.COMPILATION_ERROR,
- "Cannot enforce field \"" + String.join(".", index.getKeyFieldNames().get(i))
- + "\" to have type " + index.getKeyFieldTypes().get(i));
- }
- if (enforcedFieldType == null) {
- recordNameTypesMap.put(splits.get(splits.size() - 1),
- AUnionType.createUnknownableType(index.getKeyFieldTypes().get(i)));
- }
- enforcedRecordType = new ARecordType(lastNestedRecord.getTypeName(),
- recordNameTypesMap.keySet().toArray(new String[recordNameTypesMap.size()]),
- recordNameTypesMap.values().toArray(new IAType[recordNameTypesMap.size()]),
- lastNestedRecord.isOpen());
- // make nullable/missable if the original nestedFieldType was nullable/missable
- enforcedRecordType = keepUnknown(nestedFieldType, (ARecordType) enforcedRecordType);
- }
-
- // Create the enforced type for the nested fields in the schema, from the ground up
- if (!nestedTypeStack.isEmpty()) {
- while (!nestedTypeStack.isEmpty()) {
- Pair<IAType, String> nestedType = nestedTypeStack.pop();
- ARecordType nestedRecType = (ARecordType) TypeComputeUtils.getActualType(nestedType.first);
- IAType[] nestedRecTypeFieldTypes = nestedRecType.getFieldTypes().clone();
- nestedRecTypeFieldTypes[nestedRecType.getFieldIndex(nestedType.second)] = enforcedRecordType;
- enforcedRecordType = new ARecordType(nestedRecType.getTypeName() + "_enforced",
- nestedRecType.getFieldNames(), nestedRecTypeFieldTypes, nestedRecType.isOpen());
- // make nullable/missable if the original nestedRecType was nullable/missable
- enforcedRecordType = keepUnknown(nestedType.first, (ARecordType) enforcedRecordType);
- }
- }
+ switch (Index.IndexCategory.of(index.getIndexType())) {
+ case VALUE:
+ enforcedRecordType = appendValueIndexType(enforcedRecordType,
+ (Index.ValueIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+ break;
+ case TEXT:
+ enforcedRecordType = appendTextIndexType(enforcedRecordType,
+ (Index.TextIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+ break;
+ case ARRAY:
+ enforcedRecordType = appendArrayIndexTypes(enforcedRecordType,
+ (Index.ArrayIndexDetails) index.getIndexDetails(), enforcedTypeBuilder);
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
+ String.valueOf(index.getIndexType()));
}
}
- // the final enforcedRecordType built must be ARecordType since the original dataset rec. type can't be nullable
+
validateRecord(enforcedRecordType);
- return new Pair<>((ARecordType) enforcedRecordType, enforcedMetaType);
+ return new Pair<>(enforcedRecordType, metaType);
+ }
+
+ private static ARecordType appendValueIndexType(ARecordType enforcedRecordType,
+ Index.ValueIndexDetails valueIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+ throws AlgebricksException {
+ List<List<String>> keyFieldNames = valueIndexDetails.getKeyFieldNames();
+ List<IAType> keyFieldTypes = valueIndexDetails.getKeyFieldTypes();
+ List<Integer> keySources = valueIndexDetails.getKeyFieldSourceIndicators();
+ for (int i = 0; i < keyFieldNames.size(); i++) {
+ if (keySources.get(i) != Index.RECORD_INDICATOR) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ "Indexing an open field is only supported on the record part");
+ }
+ enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
+ Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+ validateRecord(enforcedRecordType);
+ enforcedRecordType = enforcedTypeBuilder.build();
+ }
+
+ return enforcedRecordType;
+ }
+
+ private static ARecordType appendTextIndexType(ARecordType enforcedRecordType,
+ Index.TextIndexDetails textIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+ throws AlgebricksException {
+ List<List<String>> keyFieldNames = textIndexDetails.getKeyFieldNames();
+ List<IAType> keyFieldTypes = textIndexDetails.getKeyFieldTypes();
+ List<Integer> keySources = textIndexDetails.getKeyFieldSourceIndicators();
+ for (int i = 0; i < keyFieldNames.size(); i++) {
+ if (keySources.get(i) != Index.RECORD_INDICATOR) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ "Indexing an open field is only supported on the record part");
+ }
+ enforcedTypeBuilder.reset(enforcedRecordType, keyFieldNames.get(i),
+ Collections.nCopies(keyFieldNames.get(i).size(), 0), keyFieldTypes.get(i));
+ validateRecord(enforcedRecordType);
+ enforcedRecordType = enforcedTypeBuilder.build();
+ }
+
+ return enforcedRecordType;
+ }
+
+ private static ARecordType appendArrayIndexTypes(ARecordType enforcedRecordType,
+ Index.ArrayIndexDetails arrayIndexDetails, EnforcedTypeBuilder enforcedTypeBuilder)
+ throws AlgebricksException {
+ for (Index.ArrayIndexElement e : arrayIndexDetails.getElementList()) {
+ if (e.getSourceIndicator() != Index.RECORD_INDICATOR) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR,
+ "Indexing an open field is only supported on the record part");
+ }
+ List<List<String>> unnestList = e.getUnnestList();
+ List<List<String>> projectList = e.getProjectList();
+ List<IAType> typeList = e.getTypeList();
+ for (int i = 0; i < projectList.size(); i++) {
+ List<String> project = projectList.get(i);
+ enforcedTypeBuilder.reset(enforcedRecordType,
+ ArrayIndexUtil.getFlattenedKeyFieldNames(unnestList, project),
+ ArrayIndexUtil.getArrayDepthIndicator(unnestList, project), typeList.get(i));
+ validateRecord(enforcedRecordType);
+ enforcedRecordType = enforcedTypeBuilder.build();
+ }
+ }
+
+ return enforcedRecordType;
}
/**
diff --git a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
index 5fcb6e0..f690018 100644
--- a/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
+++ b/asterixdb/asterix-metadata/src/test/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslatorTest.java
@@ -73,7 +73,7 @@
Index index = new Index(dvTest, "d1", "i1", IndexType.BTREE,
Collections.singletonList(Collections.singletonList("row_id")),
indicator == null ? null : Collections.singletonList(indicator),
- Collections.singletonList(BuiltinType.AINT64), -1, false, false, false, 0);
+ Collections.singletonList(BuiltinType.AINT64), false, false, false, 0);
MetadataNode mockMetadataNode = mock(MetadataNode.class);
when(mockMetadataNode.getDatatype(any(), any(DataverseName.class), anyString())).thenReturn(new Datatype(
@@ -85,11 +85,11 @@
ITupleReference tuple = idxTranslator.getTupleFromMetadataEntity(index);
Index deserializedIndex = idxTranslator.getMetadataEntityFromTuple(tuple);
if (indicator == null) {
- Assert.assertEquals(Collections.singletonList(new Integer(0)),
- deserializedIndex.getKeyFieldSourceIndicators());
+ Assert.assertEquals(Collections.singletonList(0),
+ ((Index.ValueIndexDetails) deserializedIndex.getIndexDetails()).getKeyFieldSourceIndicators());
} else {
- Assert.assertEquals(index.getKeyFieldSourceIndicators(),
- deserializedIndex.getKeyFieldSourceIndicators());
+ Assert.assertEquals(((Index.ValueIndexDetails) index.getIndexDetails()).getKeyFieldSourceIndicators(),
+ ((Index.ValueIndexDetails) deserializedIndex.getIndexDetails()).getKeyFieldSourceIndicators());
}
}
}
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 4ee8e83..c3a1a9c 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-om</artifactId>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index c644bbe..2dfc603 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -23,6 +23,7 @@
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
@@ -96,4 +97,15 @@
public TokenizerType getTokenizerType() {
return TokenizerType.LIST;
}
+
+ @Override
+ public TokenizerCategory getTokenizerCategory() {
+ // Currently, the AOrderedListBinaryTokenizer is used in ftcontains() function only,
+ // which means it is used to tokenize WORD only
+ //
+ // The functions that utilize NGRAM index are edit_distance(), edit_distance_check() and so on,
+ // which don't take a list as input (string only).
+ // e.g. edit_distance(u.name, "Suzanna Tilson")
+ return TokenizerCategory.WORD;
+ }
}
diff --git a/asterixdb/asterix-replication/pom.xml b/asterixdb/asterix-replication/pom.xml
index b7bae74..0a45c33 100644
--- a/asterixdb/asterix-replication/pom.xml
+++ b/asterixdb/asterix-replication/pom.xml
@@ -3,7 +3,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-replication</artifactId>
<licenses>
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 6b8bf71..19f7d8c 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-runtime</artifactId>
<properties>
@@ -85,10 +85,6 @@
</dependency>
<dependency>
<groupId>org.apache.hyracks</groupId>
- <artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
- </dependency>
- <dependency>
- <groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-dataflow-std</artifactId>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
deleted file mode 100644
index ff0a9f6..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsEvaluator.java
+++ /dev/null
@@ -1,469 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.
- */
-package org.apache.asterix.runtime.evaluators.common;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.BinaryTokenizerFactoryProvider;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.om.base.ABoolean;
-import org.apache.asterix.om.base.ANull;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringLowercaseTokenPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.data.std.util.BinaryEntry;
-import org.apache.hyracks.data.std.util.BinaryHashSet;
-import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
-import org.apache.hyracks.util.string.UTF8StringUtil;
-
-public class FullTextContainsEvaluator implements IScalarEvaluator {
-
- // assuming type indicator in serde format
- protected static final int TYPE_INDICATOR_SIZE = 1;
-
- protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- protected final DataOutput out = resultStorage.getDataOutput();
- protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
- protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
- protected TaggedValuePointable[] argOptions;
- protected final IScalarEvaluator evalLeft;
- protected final IScalarEvaluator evalRight;
- protected IScalarEvaluator[] evalOptions;
- protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
- protected IPointable outRight = VoidPointable.FACTORY.createPointable();
- protected IPointable[] outOptions;
- protected int optionArgsLength;
-
- // To conduct a full-text search, we convert all strings to the lower case.
- // In addition, since each token does not include the length information (2 bytes) in the beginning,
- // We need to have a different binary comparator that is different from a standard string comparator.
- // i.e. A token comparator that receives the length of a token as a parameter.
- private final IBinaryComparator strLowerCaseTokenCmp =
- BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_TOKEN_POINTABLE_INSTANCE.createBinaryComparator();
- private final IBinaryComparator strLowerCaseCmp =
- BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE.createBinaryComparator();
- private IBinaryTokenizer tokenizerForLeftArray = null;
- private IBinaryTokenizer tokenizerForRightArray = null;
-
- // Case insensitive hash for full-text search
- private IBinaryHashFunction hashFunc = null;
-
- // keyEntry used in the hash-set
- private BinaryEntry keyEntry = null;
-
- // Parameter: number of bucket, frame size, hashFunction, Comparator, byte
- // array that contains the key
- private BinaryHashSet rightHashSet = null;
-
- // Keeps the query array. This is used to check whether the query predicate has been changed (e.g., join case)
- private byte[] queryArray = null;
- private int queryArrayStartOffset = -1;
- private int queryArrayLength = -1;
-
- // If the following is 1, then we will do a disjunctive search.
- // Else if it is equal to the number of tokens, then we will do a conjunctive search.
- private int occurrenceThreshold = 1;
-
- static final int HASH_SET_SLOT_SIZE = 101;
- static final int HASH_SET_FRAME_SIZE = 32768;
-
- @SuppressWarnings("unchecked")
- protected ISerializerDeserializer<ABoolean> serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
- @SuppressWarnings("unchecked")
- protected ISerializerDeserializer<ANull> nullSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-
- public FullTextContainsEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext context)
- throws HyracksDataException {
- evalLeft = args[0].createScalarEvaluator(context);
- evalRight = args[1].createScalarEvaluator(context);
- optionArgsLength = args.length - 2;
- this.evalOptions = new IScalarEvaluator[optionArgsLength];
- this.outOptions = new IPointable[optionArgsLength];
- this.argOptions = new TaggedValuePointable[optionArgsLength];
- // Full-text search options
- for (int i = 0; i < optionArgsLength; i++) {
- this.evalOptions[i] = args[i + 2].createScalarEvaluator(context);
- this.outOptions[i] = VoidPointable.FACTORY.createPointable();
- this.argOptions[i] = TaggedValuePointable.FACTORY.createPointable();
- }
- }
-
- @Override
- public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- boolean isReturnNull = false;
- resultStorage.reset();
-
- evalLeft.evaluate(tuple, argLeft);
- evalRight.evaluate(tuple, argRight);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argLeft, argRight)) {
- if (result.getByteArray()[0] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- return;
- }
-
- // null value, but check other arguments for missing first (higher priority)
- isReturnNull = true;
- }
-
- argLeft.getValue(outLeft);
- argRight.getValue(outRight);
-
- for (int i = 0; i < optionArgsLength; i++) {
- evalOptions[i].evaluate(tuple, argOptions[i]);
-
- if (PointableHelper.checkAndSetMissingOrNull(result, argOptions[i])) {
- if (result.getByteArray()[0] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
- return;
- }
-
- // null value, but check other arguments for missing first (higher priority)
- isReturnNull = true;
- }
-
- argOptions[i].getValue(outOptions[i]);
- }
-
- if (isReturnNull) {
- PointableHelper.setNull(result);
- return;
- }
-
- ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
- ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
-
- // Checks whether two appropriate types are provided or not. If not, null will be written.
- if (!checkArgTypes(typeTag1, typeTag2)) {
- try {
- nullSerde.serialize(ANull.NULL, out);
- } catch (HyracksDataException e) {
- throw HyracksDataException.create(e);
- }
- result.set(resultStorage);
- return;
- }
-
- try {
- ABoolean b = fullTextContainsWithArg(typeTag2, argLeft, argRight) ? ABoolean.TRUE : ABoolean.FALSE;
- serde.serialize(b, out);
- } catch (HyracksDataException e1) {
- throw HyracksDataException.create(e1);
- }
- result.set(resultStorage);
- }
-
- /**
- * Conducts a full-text search. The basic logic is as follows.
- * 1) Tokenizes the given query predicate(s). Puts them into a hash set.
- * 2) Tokenizes the given field. For each token, checks whether the hash set contains it.
- * If so, increase foundCount for a newly found token.
- * 3) As soon as the foundCount becomes the given threshold, stops the search and returns true.
- * After traversing all tokens and still the foundCount is less than the given threshold, then returns false.
- */
- private boolean fullTextContainsWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2)
- throws HyracksDataException {
- // Since a fulltext search form is "ftcontains(X,Y,options)",
- // X (document) is the left side and Y (query predicate) is the right side.
-
- // Initialize variables that are required to conduct full-text search. (e.g., hash-set, tokenizer ...)
- if (rightHashSet == null) {
- initializeFullTextContains();
- }
-
- // Type tag checking is already done in the previous steps.
- // So we directly conduct the full-text search process.
- // The right side contains the query predicates
- byte[] arg2Array = arg2.getByteArray();
-
- // Checks whether a new query predicate is introduced.
- // If not, we can re-use the query predicate array we have already created.
- if (!partOfArrayEquals(queryArray, queryArrayStartOffset, queryArrayLength, arg2Array, arg2.getStartOffset(),
- arg2.getLength())) {
- resetQueryArrayAndRight(arg2Array, typeTag2, arg2);
- } else {
- // The query predicate remains the same. However, the count of each token should be reset to zero.
- // Here, we visit all elements to clear the count.
- rightHashSet.clearFoundCount();
- }
-
- return readLeftAndConductSearch(arg1);
- }
-
- private void initializeFullTextContains() {
- // We use a hash set to store tokens from the right side (query predicate).
- // Initialize necessary variables.
- hashFunc = new PointableBinaryHashFunctionFactory(UTF8StringLowercaseTokenPointable.FACTORY)
- .createBinaryHashFunction();
- keyEntry = new BinaryEntry();
- // Parameter: number of bucket, frame size, hashFunction, Comparator, byte array
- // that contains the key (this array will be set later.)
- rightHashSet = new BinaryHashSet(HASH_SET_SLOT_SIZE, HASH_SET_FRAME_SIZE, hashFunc, strLowerCaseTokenCmp, null);
- tokenizerForLeftArray = BinaryTokenizerFactoryProvider.INSTANCE
- .getWordTokenizerFactory(ATypeTag.STRING, false, true).createTokenizer();
- }
-
- void resetQueryArrayAndRight(byte[] arg2Array, ATypeTag typeTag2, IPointable arg2) throws HyracksDataException {
- // If the right side is an (un)ordered list, we need to apply the (un)ordered list tokenizer.
- switch (typeTag2) {
- case ARRAY:
- tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
- .getWordTokenizerFactory(ATypeTag.ARRAY, false, true).createTokenizer();
- break;
- case MULTISET:
- tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
- .getWordTokenizerFactory(ATypeTag.MULTISET, false, true).createTokenizer();
- break;
- case STRING:
- tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
- .getWordTokenizerFactory(ATypeTag.STRING, false, true).createTokenizer();
- break;
- default:
- break;
- }
-
- queryArray = arg2Array;
- queryArrayStartOffset = arg2.getStartOffset();
- queryArrayLength = arg2.getLength();
-
- // Clear hash set for the search predicates.
- rightHashSet.clear();
- rightHashSet.setRefArray(queryArray);
-
- // Token count in this query
- int queryTokenCount = 0;
- int uniqueQueryTokenCount = 0;
-
- int numBytesToStoreLength;
-
- // Reset the tokenizer for the given keywords in the given query
- if (typeTag2 == ATypeTag.STRING) {
- // How many bytes are required to store the length of the given token?
- numBytesToStoreLength = UTF8StringUtil
- .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(queryArray, queryArrayStartOffset));
- queryArrayStartOffset = queryArrayStartOffset + numBytesToStoreLength;
- queryArrayLength = queryArrayLength - numBytesToStoreLength;
- }
- tokenizerForRightArray.reset(queryArray, queryArrayStartOffset, queryArrayLength);
-
- // Create tokens from the given query predicate
- while (tokenizerForRightArray.hasNext()) {
- tokenizerForRightArray.next();
- queryTokenCount++;
-
- // Insert the starting position and the length of the current token into the hash set.
- // We don't store the actual value of this token since we can access it via offset and length.
- int tokenOffset = tokenizerForRightArray.getToken().getStartOffset();
- int tokenLength = tokenizerForRightArray.getToken().getTokenLength();
-
- // If a token comes from a string tokenizer, each token doesn't have the length data
- // in the beginning. Instead, if a token comes from an (un)ordered list, each token has
- // the length data in the beginning. Since KeyEntry keeps the length data
- // as a parameter, we need to adjust token offset and length in this case.
- // e.g., 8database <--- we only need to store the offset of 'd' and length 8.
- if (typeTag2 == ATypeTag.ARRAY || typeTag2 == ATypeTag.MULTISET) {
- // How many bytes are required to store the length of the given token?
- numBytesToStoreLength = UTF8StringUtil.getNumBytesToStoreLength(
- UTF8StringUtil.getUTFLength(tokenizerForRightArray.getToken().getData(),
- tokenizerForRightArray.getToken().getStartOffset()));
- tokenOffset = tokenOffset + numBytesToStoreLength;
- tokenLength = tokenLength - numBytesToStoreLength;
- }
- keyEntry.set(tokenOffset, tokenLength);
-
- // Check whether the given token is a phrase.
- // Currently, for the full-text search, we don't support a phrase search yet.
- // So, each query predicate should have only one token.
- // The same logic should be applied in AbstractTOccurrenceSearcher() class.
- checkWhetherFullTextPredicateIsPhrase(typeTag2, queryArray, tokenOffset, tokenLength, queryTokenCount);
-
- // Count the number of tokens in the given query. We only count the unique tokens.
- // We only care about the first insertion of the token into the hash set
- // since we apply the set semantics.
- // e.g., if a query predicate is ["database","system","database"],
- // then "database" should be counted only once.
- // Thus, when we find the current token (we don't increase the count in this case),
- // it should not exist.
- if (rightHashSet.find(keyEntry, queryArray, false) == -1) {
- rightHashSet.put(keyEntry);
- uniqueQueryTokenCount++;
- }
-
- }
-
- // Apply the full-text search option here
- // Based on the search mode option - "any" or "all", set the occurrence threshold of tokens.
- setFullTextOption(argOptions, uniqueQueryTokenCount);
- }
-
- private void checkWhetherFullTextPredicateIsPhrase(ATypeTag typeTag, byte[] refArray, int tokenOffset,
- int tokenLength, int queryTokenCount) throws HyracksDataException {
- switch (typeTag) {
- case STRING:
- if (queryTokenCount > 1) {
- throw new HyracksDataException(
- "Phrase in Full-text search is not supported. An expression should include only one word.");
- }
- break;
- case ARRAY:
- case MULTISET:
- for (int j = 0; j < tokenLength; j++) {
- if (DelimitedUTF8StringBinaryTokenizer.isSeparator((char) refArray[tokenOffset + j])) {
- throw new HyracksDataException(
- "Phrase in Full-text is not supported. An expression should include only one word."
- + (char) refArray[tokenOffset + j] + " " + refArray[tokenOffset + j]);
- }
- }
- break;
- default:
- throw new HyracksDataException("Full-text search can be only executed on STRING or (UN)ORDERED LIST.");
- }
- }
-
- /**
- * Sets the full-text options. The odd element is an option name and the even element is the argument
- * for that option. (e.g., argOptions[0] = "mode", argOptions[1] = "all")
- */
- private void setFullTextOption(IPointable[] argOptions, int uniqueQueryTokenCount) throws HyracksDataException {
- // By default, we conduct a conjunctive search.
- occurrenceThreshold = uniqueQueryTokenCount;
- for (int i = 0; i < optionArgsLength; i = i + 2) {
- // mode option
- if (compareStrInByteArrayAndPointable(FullTextContainsDescriptor.getSearchModeOptionArray(), argOptions[i],
- true) == 0) {
- if (compareStrInByteArrayAndPointable(FullTextContainsDescriptor.getDisjunctiveFTSearchOptionArray(),
- argOptions[i + 1], true) == 0) {
- // ANY
- occurrenceThreshold = 1;
- } else if (compareStrInByteArrayAndPointable(
- FullTextContainsDescriptor.getConjunctiveFTSearchOptionArray(), argOptions[i + 1], true) == 0) {
- // ALL
- occurrenceThreshold = uniqueQueryTokenCount;
- }
- }
- }
- }
-
- boolean readLeftAndConductSearch(IPointable arg1) throws HyracksDataException {
- // Now, we traverse the left side (document field) and tokenize the array and check whether each token
- // exists in the hash set. If it's the first time we find it, we increase foundCount.
- // As soon as foundCount is greater than occurrenceThreshold, we return true and stop.
- int foundCount = 0;
-
- // The left side: field (document)
- // Resets the tokenizer for the given keywords in a document.
-
- // How many bytes are required to store the length of the given string?
- int numBytesToStoreLength = UTF8StringUtil
- .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(arg1.getByteArray(), arg1.getStartOffset()));
- int startOffset = arg1.getStartOffset() + numBytesToStoreLength;
- int length = arg1.getLength() - numBytesToStoreLength;
-
- tokenizerForLeftArray.reset(arg1.getByteArray(), startOffset, length);
-
- // Creates tokens from a field in the left side (document)
- while (tokenizerForLeftArray.hasNext()) {
- tokenizerForLeftArray.next();
-
- // Records the starting position and the length of the current token.
- keyEntry.set(tokenizerForLeftArray.getToken().getStartOffset(),
- tokenizerForLeftArray.getToken().getTokenLength());
-
- // Checks whether this token exists in the query hash-set.
- // We don't count multiple occurrence of a token now.
- // So, finding the same query predicate twice will not be counted as a found.
- if (rightHashSet.find(keyEntry, arg1.getByteArray(), true) == 1) {
- foundCount++;
- if (foundCount >= occurrenceThreshold) {
- return true;
- }
- }
- }
-
- // Traversed all tokens. However, the count is not greater than the threshold.
- return false;
- }
-
- private int compareStrInByteArrayAndPointable(byte[] left, IPointable right, boolean rightTypeTagIncluded)
- throws HyracksDataException {
- int rightTypeTagLength = rightTypeTagIncluded ? 1 : 0;
-
- return strLowerCaseCmp.compare(left, 0, left.length, right.getByteArray(),
- right.getStartOffset() + rightTypeTagLength, right.getLength() - rightTypeTagLength);
- }
-
- /**
- * Checks the argument types. The argument1 should be a string.
- * The argument2 should be a string or an (un)ordered list.
- */
- protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
- if ((typeTag1 != ATypeTag.STRING) || (typeTag2 != ATypeTag.ARRAY && typeTag2 != ATypeTag.MULTISET
- && !ATypeHierarchy.isCompatible(typeTag1, typeTag2))) {
- return false;
- }
- return true;
- }
-
- /**
- * Checks whether the content of the given two arrays are equal.
- * The code is utilizing the Arrays.equals() code. The difference is that
- * this method only compares the certain portion of each array.
- */
- private static boolean partOfArrayEquals(byte[] array1, int start1, int length1, byte[] array2, int start2,
- int length2) {
- // Sanity check
- if (length1 != length2 || array1 == null || array2 == null) {
- return false;
- }
-
- if (array1 == array2 && start1 == start2 && length1 == length2) {
- return true;
- }
-
- int offset = 0;
- while (offset < length1) {
- if (array1[start1 + offset] != array2[start2 + offset]) {
- return false;
- }
- offset++;
- }
-
- return true;
- }
-
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java
new file mode 100644
index 0000000..fce2061
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/FullTextContainsFunctionEvaluator.java
@@ -0,0 +1,492 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.runtime.evaluators.common;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.BinaryTokenizerFactoryProvider;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringLowercaseTokenPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.BinaryEntry;
+import org.apache.hyracks.data.std.util.BinaryHashSet;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+public class FullTextContainsFunctionEvaluator implements IScalarEvaluator {
+
+ // assuming type indicator in serde format
+ protected static final int TYPE_INDICATOR_SIZE = 1;
+
+ protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ protected final DataOutput out = resultStorage.getDataOutput();
+ protected final TaggedValuePointable argLeft = TaggedValuePointable.FACTORY.createPointable();
+ protected final TaggedValuePointable argRight = TaggedValuePointable.FACTORY.createPointable();
+ protected TaggedValuePointable[] argOptions;
+ protected final IScalarEvaluator evalLeft;
+ protected final IScalarEvaluator evalRight;
+ protected IScalarEvaluator[] evalOptions;
+ protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
+ protected IPointable outRight = VoidPointable.FACTORY.createPointable();
+ protected IPointable[] outOptions;
+ protected int optionArgsLength;
+ // By default, we conduct a conjunctive search.
+ protected FullTextContainsFunctionDescriptor.SearchMode mode = FullTextContainsFunctionDescriptor.SearchMode.ALL;
+
+ // To conduct a full-text search, we convert all strings to the lower case.
+ // In addition, since each token does not include the length information (2 bytes) in the beginning,
+ // We need to have a different binary comparator that is different from a standard string comparator.
+ // i.e. A token comparator that receives the length of a token as a parameter.
+ private final IBinaryComparator strLowerCaseTokenCmp =
+ BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_TOKEN_POINTABLE_INSTANCE.createBinaryComparator();
+ private final IBinaryComparator strLowerCaseCmp =
+ BinaryComparatorFactoryProvider.UTF8STRING_LOWERCASE_POINTABLE_INSTANCE.createBinaryComparator();
+
+ private IFullTextConfigEvaluator ftEvaluatorLeft;
+ private IFullTextConfigEvaluator ftEvaluatorRight;
+
+ // Case insensitive hash for full-text search
+ private IBinaryHashFunction hashFunc = null;
+
+ // keyEntry used in the hash-set
+ private BinaryEntry keyEntry = null;
+
+ // Parameter: number of bucket, frame size, hashFunction, Comparator, byte
+ // array that contains the key
+ private BinaryHashSet rightHashSet = null;
+
+ // Keeps the query array. This is used to check whether the query predicate has been changed (e.g., join case)
+ private byte[] queryArray = null;
+ private int queryArrayStartOffset = -1;
+ private int queryArrayLength = -1;
+
+ // If the following is 1, then we will do a disjunctive search.
+ // Else if it is equal to the number of tokens, then we will do a conjunctive search.
+ private int occurrenceThreshold = 1;
+
+ static final int HASH_SET_SLOT_SIZE = 101;
+ static final int HASH_SET_FRAME_SIZE = 32768;
+
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ABoolean> serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ANull> nullSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+ public FullTextContainsFunctionEvaluator(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) throws HyracksDataException {
+
+ evalLeft = args[0].createScalarEvaluator(context);
+ evalRight = args[1].createScalarEvaluator(context);
+ optionArgsLength = args.length - 2;
+ this.evalOptions = new IScalarEvaluator[optionArgsLength];
+ this.outOptions = new IPointable[optionArgsLength];
+ this.argOptions = new TaggedValuePointable[optionArgsLength];
+
+ // We need to have two dedicated ftEvaluatorLeft and ftEvaluatorRight to let them have dedicated tokenizers.
+ //
+ // ftEvaluatorLeft and ftEvaluatorRight are shared by multiple threads on the NC node,
+ // so each thread needs a local copy of them here
+ this.ftEvaluatorLeft = fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+ this.ftEvaluatorRight = fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
+ for (int i = 0; i < optionArgsLength; i++) {
+ this.evalOptions[i] = args[i + 2].createScalarEvaluator(context);
+ this.outOptions[i] = VoidPointable.FACTORY.createPointable();
+ this.argOptions[i] = TaggedValuePointable.FACTORY.createPointable();
+ }
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ boolean isReturnNull = false;
+ resultStorage.reset();
+
+ evalLeft.evaluate(tuple, argLeft);
+ evalRight.evaluate(tuple, argRight);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argLeft, argRight)) {
+ if (result.getByteArray()[0] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ return;
+ }
+
+ // null value, but check other arguments for missing first (higher priority)
+ isReturnNull = true;
+ }
+
+ argLeft.getValue(outLeft);
+ argRight.getValue(outRight);
+
+ for (int i = 0; i < optionArgsLength; i++) {
+ evalOptions[i].evaluate(tuple, argOptions[i]);
+
+ if (PointableHelper.checkAndSetMissingOrNull(result, argOptions[i])) {
+ if (result.getByteArray()[0] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
+ return;
+ }
+
+ // null value, but check other arguments for missing first (higher priority)
+ isReturnNull = true;
+ }
+
+ argOptions[i].getValue(outOptions[i]);
+ }
+
+ if (isReturnNull) {
+ PointableHelper.setNull(result);
+ return;
+ }
+
+ ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+ ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
+
+ // Checks whether two appropriate types are provided or not. If not, null will be written.
+ if (!checkArgTypes(typeTag1, typeTag2)) {
+ try {
+ nullSerde.serialize(ANull.NULL, out);
+ } catch (HyracksDataException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ return;
+ }
+
+ try {
+ ABoolean b = fullTextContainsWithArg(typeTag2, argLeft, argRight) ? ABoolean.TRUE : ABoolean.FALSE;
+ serde.serialize(b, out);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(ErrorCode.ERROR_PROCESSING_TUPLE, e);
+ }
+
+ result.set(resultStorage);
+ }
+
+ /**
+ * Conducts a full-text search. The basic logic is as follows.
+ * 1) Tokenizes the given query predicate(s). Puts them into a hash set.
+ * 2) Tokenizes the given field. For each token, checks whether the hash set contains it.
+ * If so, increase foundCount for a newly found token.
+ * 3) As soon as the foundCount becomes the given threshold, stops the search and returns true.
+ * After traversing all tokens and still the foundCount is less than the given threshold, then returns false.
+ */
+ private boolean fullTextContainsWithArg(ATypeTag typeTag2, IPointable arg1, IPointable arg2)
+ throws HyracksDataException, AlgebricksException {
+ // The main logic
+
+ // Since a fulltext search form is "ftcontains(X,Y,options)",
+ // X (document) is the left side and Y (query predicate) is the right side.
+
+ setFullTextOption(argOptions);
+
+ // Initialize variables that are required to conduct full-text search. (e.g., hash-set, tokenizer ...)
+ if (rightHashSet == null) {
+ initializeFullTextContains();
+ }
+
+ // Type tag checking is already done in the previous steps.
+ // So we directly conduct the full-text search process.
+ // The right side contains the query predicates
+ byte[] arg2Array = arg2.getByteArray();
+
+ // Checks whether a new query predicate is introduced.
+ // If not, we can re-use the query predicate array we have already created.
+ if (!partOfArrayEquals(queryArray, queryArrayStartOffset, queryArrayLength, arg2Array, arg2.getStartOffset(),
+ arg2.getLength())) {
+ resetQueryArrayAndRight(arg2Array, typeTag2, arg2);
+ } else {
+ // The query predicate remains the same. However, the count of each token should be reset to zero.
+ // Here, we visit all elements to clear the count.
+ rightHashSet.clearFoundCount();
+ }
+
+ return readLeftAndConductSearch(arg1);
+ }
+
+ private void initializeFullTextContains() {
+ // We use a hash set to store tokens from the right side (query predicate).
+ // Initialize necessary variables.
+ hashFunc = new PointableBinaryHashFunctionFactory(UTF8StringLowercaseTokenPointable.FACTORY)
+ .createBinaryHashFunction();
+ keyEntry = new BinaryEntry();
+ // Parameter: number of bucket, frame size, hashFunction, Comparator, byte array
+ // that contains the key (this array will be set later.)
+ rightHashSet = new BinaryHashSet(HASH_SET_SLOT_SIZE, HASH_SET_FRAME_SIZE, hashFunc, strLowerCaseTokenCmp, null);
+ IBinaryTokenizer tokenizerForLeftArray = BinaryTokenizerFactoryProvider.INSTANCE
+ .getWordTokenizerFactory(ATypeTag.STRING, true, true).createTokenizer();
+ ftEvaluatorLeft.setTokenizer(tokenizerForLeftArray);
+ }
+
+ void resetQueryArrayAndRight(byte[] arg2Array, ATypeTag typeTag2, IPointable arg2) throws HyracksDataException {
+
+ IBinaryTokenizer tokenizerForRightArray = null;
+ // If the right side is an (un)ordered list, we need to apply the (un)ordered list tokenizer.
+ switch (typeTag2) {
+ case ARRAY:
+ tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
+ .getWordTokenizerFactory(ATypeTag.ARRAY, false, true).createTokenizer();
+ break;
+ case MULTISET:
+ tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
+ .getWordTokenizerFactory(ATypeTag.MULTISET, false, true).createTokenizer();
+ break;
+ case STRING:
+ tokenizerForRightArray = BinaryTokenizerFactoryProvider.INSTANCE
+ .getWordTokenizerFactory(ATypeTag.STRING, false, true).createTokenizer();
+ break;
+ default:
+ break;
+ }
+ ftEvaluatorRight.setTokenizer(tokenizerForRightArray);
+
+ queryArray = arg2Array;
+ queryArrayStartOffset = arg2.getStartOffset();
+ queryArrayLength = arg2.getLength();
+
+ // Clear hash set for the search predicates.
+ rightHashSet.clear();
+ rightHashSet.setRefArray(queryArray);
+
+ // Token count in this query
+ int queryTokenCount = 0;
+ int uniqueQueryTokenCount = 0;
+
+ int numBytesToStoreLength;
+
+ // Reset the tokenizer for the given keywords in the given query
+ if (typeTag2 == ATypeTag.STRING) {
+ // How many bytes are required to store the length of the given token?
+ numBytesToStoreLength = UTF8StringUtil
+ .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(queryArray, queryArrayStartOffset));
+ queryArrayStartOffset = queryArrayStartOffset + numBytesToStoreLength;
+ queryArrayLength = queryArrayLength - numBytesToStoreLength;
+ }
+ ftEvaluatorRight.reset(queryArray, queryArrayStartOffset, queryArrayLength);
+
+ // Create tokens from the given query predicate
+ while (ftEvaluatorRight.hasNext()) {
+ ftEvaluatorRight.next();
+ queryTokenCount++;
+
+ IToken token = ftEvaluatorRight.getToken();
+ // Insert the starting position and the length of the current token into the hash set.
+ // We don't store the actual value of this token since we can access it via offset and length.
+ int tokenOffset = token.getStartOffset();
+ int tokenLength = token.getTokenLength();
+
+ // If a token comes from a string tokenizer, each token doesn't have the length data
+ // in the beginning. Instead, if a token comes from an (un)ordered list, each token has
+ // the length data in the beginning. Since KeyEntry keeps the length data
+ // as a parameter, we need to adjust token offset and length in this case.
+ // e.g., 8database <--- we only need to store the offset of 'd' and length 8.
+ if (typeTag2 == ATypeTag.ARRAY || typeTag2 == ATypeTag.MULTISET) {
+ // How many bytes are required to store the length of the given token?
+ numBytesToStoreLength = UTF8StringUtil
+ .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(token.getData(), token.getStartOffset()));
+ tokenOffset = tokenOffset + numBytesToStoreLength;
+ tokenLength = tokenLength - numBytesToStoreLength;
+ }
+ keyEntry.set(tokenOffset, tokenLength);
+
+ // Check whether the given token is a phrase.
+ // Currently, for the full-text search, we don't support a phrase search yet.
+ // So, each query predicate should have only one token.
+ // The same logic should be applied in AbstractTOccurrenceSearcher() class.
+ checkWhetherFullTextPredicateIsPhrase(typeTag2, token.getData(), tokenOffset, tokenLength, queryTokenCount);
+
+ // Count the number of tokens in the given query. We only count the unique tokens.
+ // We only care about the first insertion of the token into the hash set
+ // since we apply the set semantics.
+ // e.g., if a query predicate is ["database","system","database"],
+ // then "database" should be counted only once.
+ // Thus, when we find the current token (we don't increase the count in this case),
+ // it should not exist.
+ if (rightHashSet.find(keyEntry, queryArray, false) == -1) {
+ rightHashSet.setRefArray(token.getData());
+ rightHashSet.put(keyEntry);
+ uniqueQueryTokenCount++;
+ }
+ }
+
+ // Based on the search mode option - "any" or "all", set the occurrence threshold of tokens.
+ if (mode == FullTextContainsFunctionDescriptor.SearchMode.ANY) {
+ occurrenceThreshold = 1;
+ } else {
+ occurrenceThreshold = uniqueQueryTokenCount;
+ }
+ }
+
+ private void checkWhetherFullTextPredicateIsPhrase(ATypeTag typeTag, byte[] refArray, int tokenOffset,
+ int tokenLength, int queryTokenCount) throws HyracksDataException {
+ switch (typeTag) {
+ case STRING:
+ if (queryTokenCount > 1) {
+ throw new HyracksDataException(
+ "Phrase in Full-text search is not supported. An expression should include only one word.");
+ }
+ break;
+ case ARRAY:
+ case MULTISET:
+ for (int j = 0; j < tokenLength; j++) {
+ if (DelimitedUTF8StringBinaryTokenizer.isSeparator((char) refArray[tokenOffset + j])) {
+ throw new HyracksDataException(
+ "Phrase in Full-text is not supported. An expression should include only one word."
+ + (char) refArray[tokenOffset + j] + " " + refArray[tokenOffset + j]);
+ }
+ }
+ break;
+ default:
+ throw new HyracksDataException("Full-text search can be only executed on STRING or (UN)ORDERED LIST.");
+ }
+ }
+
+ /**
+ * Sets the full-text options. The odd element is an option name and the even element is the argument
+ * for that option. (e.g., argOptions[0] = "mode", argOptions[1] = "all")
+ */
+ private void setFullTextOption(IPointable[] argOptions) throws HyracksDataException {
+ // Maybe using a JSON parser here can make things easier?
+ for (int i = 0; i < optionArgsLength; i = i + 2) {
+ // mode option
+ if (compareStrInByteArrayAndPointable(FullTextContainsFunctionDescriptor.getSearchModeOptionArray(),
+ argOptions[i], true) == 0) {
+ if (compareStrInByteArrayAndPointable(
+ FullTextContainsFunctionDescriptor.getDisjunctiveFTSearchOptionArray(), argOptions[i + 1],
+ true) == 0) {
+ // ANY
+ mode = FullTextContainsFunctionDescriptor.SearchMode.ANY;
+ } else if (compareStrInByteArrayAndPointable(
+ FullTextContainsFunctionDescriptor.getConjunctiveFTSearchOptionArray(), argOptions[i + 1],
+ true) == 0) {
+ // ALL
+ mode = FullTextContainsFunctionDescriptor.SearchMode.ALL;
+ }
+ }
+ }
+ }
+
+ boolean readLeftAndConductSearch(IPointable arg1) throws HyracksDataException {
+ // Now, we traverse the left side (document field) and tokenize the array and check whether each token
+ // exists in the hash set. If it's the first time we find it, we increase foundCount.
+ // As soon as foundCount is greater than occurrenceThreshold, we return true and stop.
+ int foundCount = 0;
+
+ // The left side: field (document)
+ // Resets the tokenizer for the given keywords in a document.
+
+ ftEvaluatorLeft.reset(arg1.getByteArray(), arg1.getStartOffset(), arg1.getLength());
+
+ // Creates tokens from a field in the left side (document)
+ while (ftEvaluatorLeft.hasNext()) {
+ ftEvaluatorLeft.next();
+
+ IToken token = ftEvaluatorLeft.getToken();
+ // Records the starting position and the length of the current token.
+ keyEntry.set(token.getStartOffset(), token.getTokenLength());
+
+ // Checks whether this token exists in the query hash-set.
+ // We don't count multiple occurrence of a token now.
+ // So, finding the same query predicate twice will not be counted as a found.
+ if (rightHashSet.find(keyEntry, token.getData(), true) == 1) {
+ foundCount++;
+ if (foundCount >= occurrenceThreshold) {
+ return true;
+ }
+ }
+ }
+
+ // Traversed all tokens. However, the count is not greater than the threshold.
+ return false;
+ }
+
+ private int compareStrInByteArrayAndPointable(byte[] left, IPointable right, boolean rightTypeTagIncluded)
+ throws HyracksDataException {
+ int rightTypeTagLength = rightTypeTagIncluded ? 1 : 0;
+
+ return strLowerCaseCmp.compare(left, 0, left.length, right.getByteArray(),
+ right.getStartOffset() + rightTypeTagLength, right.getLength() - rightTypeTagLength);
+ }
+
+ /**
+ * Checks the argument types. The argument1 should be a string.
+ * The argument2 should be a string or an (un)ordered list.
+ */
+ protected boolean checkArgTypes(ATypeTag typeTag1, ATypeTag typeTag2) throws HyracksDataException {
+ if ((typeTag1 != ATypeTag.STRING) || (typeTag2 != ATypeTag.ARRAY && typeTag2 != ATypeTag.MULTISET
+ && !ATypeHierarchy.isCompatible(typeTag1, typeTag2))) {
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Checks whether the content of the given two arrays are equal.
+ * The code is utilizing the Arrays.equals() code. The difference is that
+ * this method only compares the certain portion of each array.
+ */
+ private static boolean partOfArrayEquals(byte[] array1, int start1, int length1, byte[] array2, int start2,
+ int length2) {
+ // Sanity check
+ if (length1 != length2 || array1 == null || array2 == null) {
+ return false;
+ }
+
+ if (array1 == array2 && start1 == start2 && length1 == length2) {
+ return true;
+ }
+
+ int offset = 0;
+ while (offset < length1) {
+ if (array1[start1 + offset] != array2[start2 + offset]) {
+ return false;
+ }
+ offset++;
+ }
+
+ return true;
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java
deleted file mode 100644
index a3c3e5e..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsDescriptor.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.
- */
-
-package org.apache.asterix.runtime.evaluators.functions;
-
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.FullTextContainsEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.util.string.UTF8StringUtil;
-
-@MissingNullInOutFunction
-public class FullTextContainsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private static final long serialVersionUID = 1L;
-
- // parameter name and its type - based on the order of parameters in this map, parameters will be re-arranged.
- private static final Map<String, ATypeTag> paramTypeMap = new LinkedHashMap<>();
-
- public static final String SEARCH_MODE_OPTION = "mode";
- public static final String DISJUNCTIVE_SEARCH_MODE_OPTION = "any";
- public static final String CONJUNCTIVE_SEARCH_MODE_OPTION = "all";
-
- private static final byte[] SEARCH_MODE_OPTION_ARRAY = UTF8StringUtil.writeStringToBytes(SEARCH_MODE_OPTION);
- private static final byte[] DISJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
- UTF8StringUtil.writeStringToBytes(DISJUNCTIVE_SEARCH_MODE_OPTION);
- private static final byte[] CONJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
- UTF8StringUtil.writeStringToBytes(CONJUNCTIVE_SEARCH_MODE_OPTION);
-
- static {
- paramTypeMap.put(SEARCH_MODE_OPTION, ATypeTag.STRING);
- }
-
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new FullTextContainsDescriptor();
- }
- };
-
- /**
- * Creates full-text search evaluator. There are three arguments:
- * arg0: Expression1 - search field
- * arg1: Expression2 - search predicate
- * arg2 and so on: Full-text search option
- */
- @Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
- return new IScalarEvaluatorFactory() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new FullTextContainsEvaluator(args, ctx);
- }
- };
- }
-
- @Override
- public FunctionIdentifier getIdentifier() {
- return BuiltinFunctions.FULLTEXT_CONTAINS;
- }
-
- public static byte[] getSearchModeOptionArray() {
- return SEARCH_MODE_OPTION_ARRAY;
- }
-
- public static byte[] getDisjunctiveFTSearchOptionArray() {
- return DISJUNCTIVE_SEARCH_MODE_OPTION_ARRAY;
- }
-
- public static byte[] getConjunctiveFTSearchOptionArray() {
- return CONJUNCTIVE_SEARCH_MODE_OPTION_ARRAY;
- }
-
- public static Map<String, ATypeTag> getParamTypeMap() {
- return paramTypeMap;
- }
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java
new file mode 100644
index 0000000..c6c2eed
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsFunctionDescriptor.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.FullTextContainsFunctionEvaluator;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+// Descriptor for the ftcontains() function.
+//
+// This is different from the full-text config descriptor (FullTextConfigDescriptor)
+// which contains a tokenizer and full-text filter descriptors (AbstractFullTextFilterDescriptor).
+@MissingNullInOutFunction
+public class FullTextContainsFunctionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 2L;
+
+ // parameter name and its type - based on the order of parameters in this map, parameters will be re-arranged.
+ private static final Map<String, ATypeTag> paramTypeMap = new LinkedHashMap<>();
+
+ public static final String SEARCH_MODE_OPTION = "mode";
+ private static final byte[] SEARCH_MODE_OPTION_ARRAY = UTF8StringUtil.writeStringToBytes(SEARCH_MODE_OPTION);
+ private static final byte[] DISJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
+ UTF8StringUtil.writeStringToBytes(SearchMode.ANY.getValue());
+ private static final byte[] CONJUNCTIVE_SEARCH_MODE_OPTION_ARRAY =
+ UTF8StringUtil.writeStringToBytes(SearchMode.ALL.getValue());
+
+ public enum SearchMode {
+ ANY("any"),
+ ALL("all");
+
+ private String value;
+
+ SearchMode(String value) {
+ this.value = value;
+ }
+
+ public String getValue() {
+ return value;
+ }
+ }
+
+ public static final String FULLTEXT_CONFIG_OPTION = "config";
+ private IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
+
+ static {
+ paramTypeMap.put(SEARCH_MODE_OPTION, ATypeTag.STRING);
+ paramTypeMap.put(FULLTEXT_CONFIG_OPTION, ATypeTag.STRING);
+ }
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new FullTextContainsFunctionDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return new FunctionTypeInferers.FullTextContainsTypeInferer();
+ }
+ };
+
+ public FullTextContainsFunctionDescriptor() {
+ }
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ super.setImmutableStates(states);
+
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory = (IFullTextConfigEvaluatorFactory) states[0];
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
+ }
+
+ /**
+ * Creates full-text search evaluator. There are three arguments:
+ * arg0: Expression1 - search field
+ * arg1: Expression2 - search predicate
+ * arg2 and so on: Full-text search option
+ */
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+ throws AlgebricksException {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 2L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+ return new FullTextContainsFunctionEvaluator(args, ctx, fullTextConfigEvaluatorFactory);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.FULLTEXT_CONTAINS;
+ }
+
+ public static byte[] getSearchModeOptionArray() {
+ return SEARCH_MODE_OPTION_ARRAY;
+ }
+
+ public static byte[] getDisjunctiveFTSearchOptionArray() {
+ return DISJUNCTIVE_SEARCH_MODE_OPTION_ARRAY;
+ }
+
+ public static byte[] getConjunctiveFTSearchOptionArray() {
+ return CONJUNCTIVE_SEARCH_MODE_OPTION_ARRAY;
+ }
+
+ public static Map<String, ATypeTag> getParamTypeMap() {
+ return paramTypeMap;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java
deleted file mode 100644
index a9197f2..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionDescriptor.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you 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.
- */
-
-package org.apache.asterix.runtime.evaluators.functions;
-
-import org.apache.asterix.common.annotations.MissingNullInOutFunction;
-import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.functions.IFunctionDescriptor;
-import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
-import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
-import org.apache.asterix.runtime.evaluators.common.FullTextContainsEvaluator;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-@MissingNullInOutFunction
-public class FullTextContainsWithoutOptionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
- private static final long serialVersionUID = 1L;
-
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- @Override
- public IFunctionDescriptor createFunctionDescriptor() {
- return new FullTextContainsWithoutOptionDescriptor();
- }
- };
-
- /**
- * Creates full-text search evaluator. There are two arguments:
- * arg0: Expression1 - search field
- * arg1: Expression2 - search predicate
- */
- @Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
- return new IScalarEvaluatorFactory() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
- return new FullTextContainsEvaluator(args, ctx);
- }
- };
- }
-
- @Override
- public FunctionIdentifier getIdentifier() {
- return BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION;
- }
-
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java
new file mode 100644
index 0000000..3263656
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/FullTextContainsWithoutOptionFunctionDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.common.annotations.MissingNullInOutFunction;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+// Descriptor for the ftcontains() function.
+//
+// This is different from the full-text config descriptor (FullTextConfigDescriptor)
+// which contains a tokenizer and full-text filter descriptors (AbstractFullTextFilterDescriptor).
+@MissingNullInOutFunction
+public class FullTextContainsWithoutOptionFunctionDescriptor extends FullTextContainsFunctionDescriptor {
+ private static final long serialVersionUID = 2L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.FULLTEXT_CONTAINS_WO_OPTION;
+ }
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new FullTextContainsWithoutOptionFunctionDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return new FunctionTypeInferers.FullTextContainsTypeInferer();
+ }
+ };
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
new file mode 100644
index 0000000..0179c34
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/AbstractFullTextFilterDescriptor.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import org.apache.asterix.common.metadata.DataverseName;
+
+public abstract class AbstractFullTextFilterDescriptor implements IFullTextFilterDescriptor {
+ protected final DataverseName dataverseName;
+ protected final String name;
+
+ public AbstractFullTextFilterDescriptor(DataverseName dataverseName, String name) {
+ this.dataverseName = dataverseName;
+ this.name = name;
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java
new file mode 100644
index 0000000..6df92a1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/FullTextConfigDescriptor.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+// Full-text config that contains a tokenizer (e.g. a WORK tokenizer) and multiple full-text filters (e.g. stopwords filter)
+// to tokenize and process tokens of full-text documents
+// When running the ftcontains() function, the full-text config can be used with or without a full-text index
+public class FullTextConfigDescriptor implements IFullTextConfigDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final DataverseName dataverseName;
+ private final String name;
+ private final TokenizerCategory tokenizerCategory;
+ private final ImmutableList<String> filterNames;
+
+ public FullTextConfigDescriptor(DataverseName dataverseName, String name, TokenizerCategory tokenizerCategory,
+ ImmutableList<String> filterNames) {
+ this.dataverseName = dataverseName;
+ this.name = name;
+ this.tokenizerCategory = tokenizerCategory;
+ this.filterNames = filterNames;
+ }
+
+ // This built-in default full-text config will be used only when no full-text config is specified by the user.
+ // Note that the default ft config descriptor is not stored in metadata catalog,
+ // and if we are trying to get a full-text config descriptor with a name of null or empty string,
+ // the metadata manager will return this default full-text config without looking into the metadata catalog
+ // In this way we avoid the edge cases to insert or delete the default config in the metadata catalog
+ public static FullTextConfigDescriptor getDefaultFullTextConfig() {
+ return new FullTextConfigDescriptor(null, null, TokenizerCategory.WORD, ImmutableList.of());
+ }
+
+ public DataverseName getDataverseName() {
+ return dataverseName;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ // We need to exclude the full-text filter descriptors from the full-text config because both of them
+ // would be in the metadata cache, that means they should be immutable to guarantee consistency
+ // So we decide to let the caller to be responsible for fetching the filter descriptors from metadata,
+ // and pass the filters as an argument here
+ //
+ // Use the util function org.apache.asterix.metadata.utils.FullTextUtil.fetchFilterAndCreateConfigEvaluator()
+ // to fetch filters according to the filter names and create full-text config evaluator
+ @Override
+ public IFullTextConfigEvaluatorFactory createEvaluatorFactory(
+ ImmutableList<AbstractFullTextFilterDescriptor> filterDescriptors) {
+ ImmutableList.Builder<IFullTextFilterEvaluatorFactory> filtersBuilder = new ImmutableList.Builder<>();
+ for (IFullTextFilterDescriptor filterDescriptor : filterDescriptors) {
+ filtersBuilder.add(filterDescriptor.createEvaluatorFactory());
+ }
+
+ return new FullTextConfigEvaluatorFactory(name, tokenizerCategory, filtersBuilder.build());
+ }
+
+ @Override
+ public TokenizerCategory getTokenizerCategory() {
+ return tokenizerCategory;
+ }
+
+ @Override
+ public ImmutableList<String> getFilterNames() {
+ return filterNames;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java
new file mode 100644
index 0000000..97f00c3
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextConfigDescriptor.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
+
+import com.google.common.collect.ImmutableList;
+
+public interface IFullTextConfigDescriptor extends Serializable {
+ String getName();
+
+ TokenizerCategory getTokenizerCategory();
+
+ ImmutableList<String> getFilterNames();
+
+ IFullTextConfigEvaluatorFactory createEvaluatorFactory(
+ ImmutableList<AbstractFullTextFilterDescriptor> filterDescriptors);
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java
new file mode 100644
index 0000000..630c8e8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/IFullTextFilterDescriptor.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+
+// Full-text filter descriptor that contains all the necessary information in the compile-time.
+// After compilation, a filter evaluator factory will be created at run-time,
+// and then a filter evaluator will be produced by the evaluator factory to process tokens after tokenization.
+//
+// Note that the filter concepts (descriptor, evaluator factory and evaluator)
+// are wrapped in the full-text config concepts accordingly.
+// The design of the filter is never to be called directly but via the full-text config.
+public interface IFullTextFilterDescriptor extends Serializable {
+ String getName();
+
+ FullTextFilterType getFilterType();
+
+ IFullTextFilterEvaluatorFactory createEvaluatorFactory();
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java
new file mode 100644
index 0000000..1dbef70
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/fulltext/StopwordsFullTextFilterDescriptor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.asterix.runtime.fulltext;
+
+import java.util.List;
+
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextFilterType;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextFilterEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.StopwordsFullTextFilterEvaluatorFactory;
+
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterDescriptor extends AbstractFullTextFilterDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public ImmutableList<String> stopwordList;
+
+ public StopwordsFullTextFilterDescriptor(DataverseName dataverseName, String name,
+ ImmutableList<String> stopwordList) {
+ super(dataverseName, name);
+ this.stopwordList = stopwordList;
+ }
+
+ @Override
+ public FullTextFilterType getFilterType() {
+ return FullTextFilterType.STOPWORDS;
+ }
+
+ public List<String> getStopwordList() {
+ return this.stopwordList;
+ }
+
+ @Override
+ public IFullTextFilterEvaluatorFactory createEvaluatorFactory() {
+ return new StopwordsFullTextFilterEvaluatorFactory(name, stopwordList);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index e1c9164..18d55aa 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -333,8 +333,8 @@
import org.apache.asterix.runtime.evaluators.functions.CreateUUIDDescriptor;
import org.apache.asterix.runtime.evaluators.functions.DecodeDataverseNameDescriptor;
import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionFunctionDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetJobParameterByNameDescriptor;
import org.apache.asterix.runtime.evaluators.functions.GetTypeDescriptor;
@@ -1080,8 +1080,8 @@
fc.add(LineRectanglePolygonAccessor.FACTORY);
// full-text function
- fc.add(FullTextContainsDescriptor.FACTORY);
- fc.add(FullTextContainsWithoutOptionDescriptor.FACTORY);
+ fc.add(FullTextContainsFunctionDescriptor.FACTORY);
+ fc.add(FullTextContainsWithoutOptionFunctionDescriptor.FACTORY);
// Record functions.
fc.add(GetRecordFieldsDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index 2af4de5..3882222 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -317,6 +317,16 @@
}
}
+ public static final class FullTextContainsTypeInferer implements IFunctionTypeInferer {
+ @Override
+ public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+ CompilerProperties compilerProps) throws AlgebricksException {
+ AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+ // get the full-text config evaluator from the expr which is set in FullTextContainsParameterCheckAndSetRule
+ fd.setImmutableStates(funcExpr.getOpaqueParameters()[0]);
+ }
+ }
+
private static IAType[] getArgumentsTypes(AbstractFunctionCallExpression funExp, IVariableTypeEnvironment ctx)
throws AlgebricksException {
IAType[] argsTypes = new IAType[funExp.getArguments().size()];
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java
new file mode 100644
index 0000000..ae03be2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+
+public class LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor
+ extends LSMTreeIndexInsertUpdateDeleteOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+ private final List<AlgebricksPipeline> secondaryKeysPipeline;
+
+ public LSMSecondaryInsertDeleteWithNestedPlanOperatorDescriptor(JobSpecification spec, RecordDescriptor outRecDesc,
+ int[] fieldPermutation, IndexOperation op, IIndexDataflowHelperFactory indexHelperFactory,
+ IModificationOperationCallbackFactory modCallbackFactory, List<AlgebricksPipeline> secondaryKeysPipeline) {
+ super(spec, outRecDesc, indexHelperFactory, fieldPermutation, op, modCallbackFactory, null);
+ this.secondaryKeysPipeline = secondaryKeysPipeline;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ return new LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable(ctx, partition, fieldPermutation,
+ inputRecDesc, op, indexHelperFactory, modCallbackFactory, secondaryKeysPipeline);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
new file mode 100644
index 0000000..0b36774
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.meta.PipelineAssembler;
+import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
+
+public class LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable
+ extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
+ private final NestedTupleSourceRuntime[] startOfPipelines;
+ private final int numberOfPrimaryKeyAndFilterFields;
+
+ public LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable(IHyracksTaskContext ctx, int partition,
+ int[] fieldPermutation, RecordDescriptor inputRecDesc, IndexOperation op,
+ IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
+ List<AlgebricksPipeline> secondaryKeysPipeline) throws HyracksDataException {
+ super(ctx, partition, indexHelperFactory, fieldPermutation, inputRecDesc, op, modCallbackFactory, null);
+ this.numberOfPrimaryKeyAndFilterFields = fieldPermutation.length;
+
+ // Build our pipeline.
+ startOfPipelines = new NestedTupleSourceRuntime[secondaryKeysPipeline.size()];
+ PipelineAssembler[] pipelineAssemblers = new PipelineAssembler[secondaryKeysPipeline.size()];
+ for (int p = 0; p < secondaryKeysPipeline.size(); p++) {
+ AlgebricksPipeline pipeline = secondaryKeysPipeline.get(p);
+ RecordDescriptor lastRecordDescriptorInPipeline =
+ pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1];
+
+ IFrameWriter outputWriter;
+ if (p == 0) {
+ // Primary pipeline (the first). Here we append the PK, filters to the final variable.
+ outputWriter = new IndexTupleInsertDelete(lastRecordDescriptorInPipeline);
+
+ } else {
+ IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, p);
+ if (outputPushRuntime == null) {
+ throw new RuntimeDataException(ErrorCode.ILLEGAL_STATE, "Invalid pipeline.");
+ }
+ outputPushRuntime.setInputRecordDescriptor(0, lastRecordDescriptorInPipeline);
+ outputWriter = outputPushRuntime;
+ }
+
+ PipelineAssembler pipelineAssembler =
+ new PipelineAssembler(pipeline, 1, 1, inputRecDesc, lastRecordDescriptorInPipeline);
+ startOfPipelines[p] = (NestedTupleSourceRuntime) pipelineAssembler.assemblePipeline(outputWriter, ctx);
+ pipelineAssemblers[p] = pipelineAssembler;
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ // Pass the input to our pipeline. The last operator in this pipeline will pass all of the tuples
+ // to our LSM accessor.
+ tuple.reset(accessor, i);
+
+ for (NestedTupleSourceRuntime nts : startOfPipelines) {
+ nts.writeTuple(buffer, i);
+ }
+
+ int n = 0;
+ try {
+ for (; n < startOfPipelines.length; n++) {
+ NestedTupleSourceRuntime nts = startOfPipelines[n];
+ try {
+ nts.open();
+ } catch (Exception e) {
+ nts.fail();
+ throw e;
+ }
+ }
+ } finally {
+ for (int j = n - 1; j >= 0; j--) {
+ startOfPipelines[j].close();
+ }
+ }
+ }
+
+ // No partial flushing was necessary. Forward entire frame.
+ writeBuffer.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+ FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+ }
+
+ private class IndexTupleInsertDelete implements IFrameWriter {
+ private final RecordDescriptor inputRecordDescriptor;
+ private FrameTupleAccessor endOfPipelineTupleAccessor;
+
+ // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
+ private ArrayTupleBuilder arrayTupleBuilder;
+ private ArrayTupleReference arrayTupleReference;
+
+ private IndexTupleInsertDelete(RecordDescriptor recordDescriptor) {
+ this.inputRecordDescriptor = recordDescriptor;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+
+ endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+ arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
+ arrayTupleReference = new ArrayTupleReference();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ILSMIndexAccessor workingLSMAccessor = (ILSMIndexAccessor) indexAccessor;
+
+ endOfPipelineTupleAccessor.reset(buffer);
+ int nTuple = endOfPipelineTupleAccessor.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+
+ // First, add the secondary keys.
+ arrayTupleBuilder.reset();
+ int nFields = endOfPipelineTupleAccessor.getFieldCount();
+ for (int f = 0; f < nFields; f++) {
+ arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
+ }
+
+ // Next, add the primary keys and filter fields.
+ for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
+ arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
+ }
+
+ // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
+ arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+ if (op.equals(IndexOperation.INSERT)) {
+ workingLSMAccessor.forceInsert(arrayTupleReference);
+ } else {
+ workingLSMAccessor.forceDelete(arrayTupleReference);
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
index df658b6..a4b4012 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorDescriptor.java
@@ -35,8 +35,8 @@
private static final long serialVersionUID = 1L;
private final int[] prevValuePermutation;
- private final int upsertIndiatorFieldIndex;
- private final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
+ protected final int upsertIndicatorFieldIndex;
+ protected final IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory;
public LSMSecondaryUpsertOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
@@ -46,7 +46,7 @@
super(spec, outRecDesc, fieldPermutation, IndexOperation.UPSERT, indexHelperFactory, tupleFilterFactory, false,
modificationOpCallbackFactory);
this.prevValuePermutation = prevValuePermutation;
- this.upsertIndiatorFieldIndex = upsertIndicatorFieldIndex;
+ this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
this.upsertIndicatorInspectorFactory = upsertIndicatorInspectorFactory;
}
@@ -55,7 +55,7 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
RecordDescriptor intputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new LSMSecondaryUpsertOperatorNodePushable(ctx, partition, indexHelperFactory, modCallbackFactory,
- tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndiatorFieldIndex,
+ tupleFilterFactory, fieldPermutation, intputRecDesc, upsertIndicatorFieldIndex,
upsertIndicatorInspectorFactory, prevValuePermutation);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index 35ae904..b588323 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -60,12 +60,13 @@
public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
- private final int upsertIndicatorFieldIndex;
- private final IBinaryBooleanInspector upsertIndicatorInspector;
private final int numberOfFields;
- private AbstractIndexModificationOperationCallback abstractModCallback;
private final boolean isPrimaryKeyIndex;
+ protected final int upsertIndicatorFieldIndex;
+ protected final IBinaryBooleanInspector upsertIndicatorInspector;
+ protected AbstractIndexModificationOperationCallback abstractModCallback;
+
public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
ITupleFilterFactory tupleFilterFactory, int[] fieldPermutation, RecordDescriptor inputRecDesc,
@@ -76,7 +77,7 @@
this.prevValueTuple.setFieldPermutation(prevValuePermutation);
this.upsertIndicatorFieldIndex = upsertIndicatorFieldIndex;
this.upsertIndicatorInspector = upsertIndicatorInspectorFactory.createBinaryBooleanInspector(ctx);
- this.numberOfFields = prevValuePermutation.length;
+ this.numberOfFields = fieldPermutation.length;
// a primary key index only has primary keys, and thus these two permutations are the same
this.isPrimaryKeyIndex = Arrays.equals(fieldPermutation, prevValuePermutation);
}
@@ -137,7 +138,12 @@
FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
}
- private boolean hasNullOrMissing(PermutingFrameTupleReference tuple) {
+ private static boolean isNullOrMissing(FrameTupleReference tuple, int fieldIdx) {
+ return TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
+ || TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ }
+
+ protected static boolean hasNullOrMissing(FrameTupleReference tuple) {
int fieldCount = tuple.getFieldCount();
for (int i = 0; i < fieldCount; i++) {
if (isNullOrMissing(tuple, i)) {
@@ -146,9 +152,4 @@
}
return false;
}
-
- private static boolean isNullOrMissing(PermutingFrameTupleReference tuple, int fieldIdx) {
- return TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
- || TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
- }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
new file mode 100644
index 0000000..d077987
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorDescriptor.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+
+public class LSMSecondaryUpsertWithNestedPlanOperatorDescriptor extends LSMSecondaryUpsertOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final List<AlgebricksPipeline> secondaryKeysPipeline;
+ private final List<AlgebricksPipeline> prevSecondaryKeysPipeline;
+
+ public LSMSecondaryUpsertWithNestedPlanOperatorDescriptor(JobSpecification spec, RecordDescriptor outRecDesc,
+ int[] fieldPermutation, IIndexDataflowHelperFactory indexHelperFactory,
+ IModificationOperationCallbackFactory modCallbackFactory, int upsertIndicatorFieldIndex,
+ IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
+ List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline) {
+ super(spec, outRecDesc, fieldPermutation, indexHelperFactory, null, modCallbackFactory,
+ upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+ this.secondaryKeysPipeline = secondaryKeysPipeline;
+ this.prevSecondaryKeysPipeline = prevSecondaryKeysPipeline;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ return new LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(ctx, partition, indexHelperFactory,
+ modCallbackFactory, fieldPermutation, inputRecDesc, upsertIndicatorFieldIndex,
+ upsertIndicatorInspectorFactory, secondaryKeysPipeline, prevSecondaryKeysPipeline);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
new file mode 100644
index 0000000..f1af496
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.asterix.runtime.operators;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback;
+import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.operators.meta.PipelineAssembler;
+import org.apache.hyracks.algebricks.runtime.operators.std.NestedTupleSourceRuntimeFactory.NestedTupleSourceRuntime;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class LSMSecondaryUpsertWithNestedPlanOperatorNodePushable extends LSMSecondaryUpsertOperatorNodePushable {
+ private final NestedTupleSourceRuntime[] startOfNewKeyPipelines;
+ private final NestedTupleSourceRuntime[] startOfPrevKeyPipelines;
+ private final int numberOfPrimaryKeyAndFilterFields;
+
+ public LSMSecondaryUpsertWithNestedPlanOperatorNodePushable(IHyracksTaskContext ctx, int partition,
+ IIndexDataflowHelperFactory indexHelperFactory, IModificationOperationCallbackFactory modCallbackFactory,
+ int[] fieldPermutation, RecordDescriptor inputRecDesc, int upsertIndicatorFieldIndex,
+ IBinaryBooleanInspectorFactory upsertIndicatorInspectorFactory,
+ List<AlgebricksPipeline> secondaryKeysPipeline, List<AlgebricksPipeline> prevSecondaryKeysPipeline)
+ throws HyracksDataException {
+ super(ctx, partition, indexHelperFactory, modCallbackFactory, null, fieldPermutation, inputRecDesc,
+ upsertIndicatorFieldIndex, upsertIndicatorInspectorFactory, null);
+ this.numberOfPrimaryKeyAndFilterFields = fieldPermutation.length;
+ this.startOfNewKeyPipelines = buildStartOfPipelines(secondaryKeysPipeline, inputRecDesc, false);
+ this.startOfPrevKeyPipelines = buildStartOfPipelines(prevSecondaryKeysPipeline, inputRecDesc, true);
+ }
+
+ private NestedTupleSourceRuntime[] buildStartOfPipelines(List<AlgebricksPipeline> pipelines,
+ RecordDescriptor inputRecordDescriptor, boolean isPrev) throws HyracksDataException {
+ NestedTupleSourceRuntime[] resultant = new NestedTupleSourceRuntime[pipelines.size()];
+ PipelineAssembler[] pipelineAssemblers = new PipelineAssembler[pipelines.size()];
+ for (int p = 0; p < pipelines.size(); p++) {
+ AlgebricksPipeline pipeline = pipelines.get(p);
+ RecordDescriptor lastRecordDescriptorInPipeline =
+ pipeline.getRecordDescriptors()[pipeline.getRecordDescriptors().length - 1];
+
+ IFrameWriter outputWriter;
+ if (p == 0) {
+ // Primary pipeline (the first). Here we perform the insert / delete.
+ outputWriter = new IndexTupleUnconditionalOperation(lastRecordDescriptorInPipeline, !isPrev);
+
+ } else {
+ IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, p);
+ if (outputPushRuntime == null) {
+ throw new RuntimeDataException(ErrorCode.ILLEGAL_STATE, "No output runtime factories found.");
+ }
+ outputPushRuntime.setInputRecordDescriptor(0, lastRecordDescriptorInPipeline);
+ outputWriter = outputPushRuntime;
+ }
+
+ PipelineAssembler pipelineAssembler =
+ new PipelineAssembler(pipeline, 1, 1, inputRecordDescriptor, lastRecordDescriptorInPipeline);
+ resultant[p] = (NestedTupleSourceRuntime) pipelineAssembler.assemblePipeline(outputWriter, ctx);
+ pipelineAssemblers[p] = pipelineAssembler;
+ }
+
+ return resultant;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ super.open();
+ frameTuple = new FrameTupleReference();
+ abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ tuple.reset(accessor, i);
+
+ // Delete all of our old keys.
+ writeTupleToPipelineStarts(buffer, i, startOfPrevKeyPipelines);
+
+ // Insert all of our new keys, if the PIDX operation was also an UPSERT (and not just a DELETE).
+ frameTuple.reset(accessor, i);
+ if (upsertIndicatorInspector.getBooleanValue(frameTuple.getFieldData(upsertIndicatorFieldIndex),
+ frameTuple.getFieldStart(upsertIndicatorFieldIndex),
+ frameTuple.getFieldLength(upsertIndicatorFieldIndex))) {
+ writeTupleToPipelineStarts(buffer, i, startOfNewKeyPipelines);
+ }
+ }
+
+ // No partial flushing was necessary. Forward entire frame.
+ writeBuffer.ensureFrameSize(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+ FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+ }
+
+ private void writeTupleToPipelineStarts(ByteBuffer buffer, int tupleIndex,
+ NestedTupleSourceRuntime[] pipelineStarts) throws HyracksDataException {
+ for (NestedTupleSourceRuntime nts : pipelineStarts) {
+ nts.writeTuple(buffer, tupleIndex);
+ }
+
+ int n = 0;
+ try {
+ for (; n < pipelineStarts.length; n++) {
+ NestedTupleSourceRuntime nts = pipelineStarts[n];
+ try {
+ nts.open();
+ } catch (Exception e) {
+ nts.fail();
+ throw e;
+ }
+ }
+ } finally {
+ for (int j = n - 1; j >= 0; j--) {
+ pipelineStarts[j].close();
+ }
+ }
+ }
+
+ private class IndexTupleUnconditionalOperation implements IFrameWriter {
+ private final RecordDescriptor inputRecordDescriptor;
+ private final boolean isInsert; // If this is not an insert, then our operation is delete.
+
+ private FrameTupleAccessor endOfPipelineTupleAccessor;
+ private FrameTupleReference endOfPipelineTupleReference;
+
+ // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
+ private ArrayTupleBuilder arrayTupleBuilder;
+ private ArrayTupleReference arrayTupleReference;
+
+ private IndexTupleUnconditionalOperation(RecordDescriptor recordDescriptor, boolean isInsert) {
+ this.inputRecordDescriptor = recordDescriptor;
+ this.isInsert = isInsert;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+ endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+ endOfPipelineTupleReference = new FrameTupleReference();
+ arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
+ arrayTupleReference = new ArrayTupleReference();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ILSMIndexAccessor workingLSMAccessor = (ILSMIndexAccessor) indexAccessor;
+
+ endOfPipelineTupleAccessor.reset(buffer);
+ int nTuple = endOfPipelineTupleAccessor.getTupleCount();
+ for (int t = 0; t < nTuple; t++) {
+
+ endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+ if (hasNullOrMissing(endOfPipelineTupleReference)) {
+ // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
+ continue;
+ }
+
+ // First, add the secondary keys.
+ arrayTupleBuilder.reset();
+ int nFields = endOfPipelineTupleAccessor.getFieldCount();
+ for (int f = 0; f < nFields; f++) {
+ arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
+ }
+
+ // Next, add the primary keys and filter fields.
+ for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
+ arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
+ }
+
+ // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
+ arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+ if (this.isInsert) {
+ abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.INSERT);
+ workingLSMAccessor.forceInsert(arrayTupleReference);
+ } else {
+ abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.DELETE);
+ workingLSMAccessor.forceDelete(arrayTupleReference);
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ }
+ }
+}
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index ce943ad..a0ac43d 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-spidersilk/pom.xml b/asterixdb/asterix-spidersilk/pom.xml
index 8933b07..c834acb 100644
--- a/asterixdb/asterix-spidersilk/pom.xml
+++ b/asterixdb/asterix-spidersilk/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/asterixdb/asterix-test-framework/pom.xml b/asterixdb/asterix-test-framework/pom.xml
index d76ee36..1b893ec 100644
--- a/asterixdb/asterix-test-framework/pom.xml
+++ b/asterixdb/asterix-test-framework/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-test-framework</artifactId>
diff --git a/asterixdb/asterix-tools/pom.xml b/asterixdb/asterix-tools/pom.xml
index 6203ace..c20acb8 100644
--- a/asterixdb/asterix-tools/pom.xml
+++ b/asterixdb/asterix-tools/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-tools</artifactId>
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 511b357..10f166e 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>apache-asterixdb</artifactId>
<groupId>org.apache.asterix</groupId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
</parent>
<artifactId>asterix-transactions</artifactId>
<licenses>
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index a930643..979d758 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -21,7 +21,7 @@
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.asterix</groupId>
<artifactId>apache-asterixdb</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
<packaging>pom</packaging>
<url>${implementation.url}</url>
@@ -80,8 +80,8 @@
<pytestlib.stage>none</pytestlib.stage>
<!-- Versions under dependencymanagement or used in many projects via properties -->
- <algebricks.version>0.3.6-SNAPSHOT</algebricks.version>
- <hyracks.version>0.3.6-SNAPSHOT</hyracks.version>
+ <algebricks.version>0.3.7-SNAPSHOT</algebricks.version>
+ <hyracks.version>0.3.7-SNAPSHOT</hyracks.version>
<hadoop.version>2.8.5</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
<log4j.version>2.14.1</log4j.version>
diff --git a/asterixdb/src/main/licenses/templates/3rdpartylicenses.txt b/asterixdb/src/main/licenses/templates/3rdpartylicenses.txt
index 8c67061..5195add 100644
--- a/asterixdb/src/main/licenses/templates/3rdpartylicenses.txt
+++ b/asterixdb/src/main/licenses/templates/3rdpartylicenses.txt
@@ -1,4 +1,179 @@
-codemirror@5.39.0
+@angular-devkit/build-angular
+MIT
+The MIT License
+
+Copyright (c) 2017 Google, Inc.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+
+@angular/animations
+MIT
+
+@angular/cdk
+MIT
+The MIT License
+
+Copyright (c) 2020 Google LLC.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+
+@angular/common
+MIT
+
+@angular/core
+MIT
+
+@angular/forms
+MIT
+
+@angular/material
+MIT
+The MIT License
+
+Copyright (c) 2020 Google LLC.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+
+@angular/platform-browser
+MIT
+
+@babel/runtime
+MIT
+MIT License
+
+Copyright (c) 2014-present Sebastian McKenzie and other contributors
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+
+@ngrx/effects
+MIT
+
+@ngrx/store
+MIT
+
+@ngrx/store-devtools
+MIT
+
+@swimlane/ngx-charts
+MIT
+MIT License
+
+Copyright (c) 2017 Swimlane
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+
+@swimlane/ngx-graph
+MIT
+(The MIT License)
+
+Copyright (c) 2016 Swimlane <info@swimlane.com>
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+'Software'), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+codemirror
MIT
MIT License
@@ -22,9 +197,10 @@
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.
-core-js@2.5.7
+
+core-js
MIT
-Copyright (c) 2014-2018 Denis Pushkarev
+Copyright (c) 2014-2020 Denis Pushkarev
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
@@ -44,11 +220,723 @@
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.
-zone.js@0.8.26
+
+css-loader
+MIT
+Copyright JS Foundation and other contributors
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+'Software'), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED 'AS IS', WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT.
+IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY
+CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT,
+TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
+SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+
+d3-array
+BSD-3-Clause
+Copyright 2010-2020 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-brush
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-collection
+BSD-3-Clause
+Copyright 2010-2016, Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-color
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-dispatch
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-drag
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-ease
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+Copyright 2001 Robert Penner
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-force
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-format
+BSD-3-Clause
+Copyright 2010-2015 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-hierarchy
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-interpolate
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-path
+BSD-3-Clause
+Copyright 2015-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-quadtree
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-scale
+BSD-3-Clause
+Copyright 2010-2015 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-selection
+BSD-3-Clause
+Copyright (c) 2010-2018, Michael Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* The name Michael Bostock may not be used to endorse or promote products
+ derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT,
+INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
+OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-shape
+BSD-3-Clause
+Copyright 2010-2015 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-time
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-time-format
+BSD-3-Clause
+Copyright 2010-2017 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-timer
+BSD-3-Clause
+Copyright 2010-2016 Mike Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+d3-transition
+BSD-3-Clause
+Copyright (c) 2010-2015, Michael Bostock
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are met:
+
+* Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+* Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+* The name Michael Bostock may not be used to endorse or promote products
+ derived from this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
+AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
+IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL MICHAEL BOSTOCK BE LIABLE FOR ANY DIRECT,
+INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING,
+BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
+OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING
+NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE,
+EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+TERMS OF USE - EASING EQUATIONS
+
+Open source under the BSD License.
+
+Copyright 2001 Robert Penner
+All rights reserved.
+
+Redistribution and use in source and binary forms, with or without modification,
+are permitted provided that the following conditions are met:
+
+- Redistributions of source code must retain the above copyright notice, this
+ list of conditions and the following disclaimer.
+
+- Redistributions in binary form must reproduce the above copyright notice,
+ this list of conditions and the following disclaimer in the documentation
+ and/or other materials provided with the distribution.
+
+- Neither the name of the author nor the names of contributors may be used to
+ endorse or promote products derived from this software without specific prior
+ written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
+ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
+WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
+DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR
+ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
+(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
+LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
+ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
+SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+
+dagre
+MIT
+Copyright (c) 2012-2014 Chris Pettitt
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
+
+
+file-saver
MIT
The MIT License
-Copyright (c) 2016-2018 Google, Inc.
+Copyright © 2016 [Eli Grey][1].
+
+Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+ [1]: http://eligrey.com
+
+
+graphlib
+MIT
+Copyright (c) 2012-2014 Chris Pettitt
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
@@ -68,7 +956,8 @@
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.
-hammerjs@2.0.8
+
+hammerjs
MIT
The MIT License (MIT)
@@ -92,71 +981,291 @@
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.
-@angular/core@6.0.7
+
+lodash
MIT
+Copyright OpenJS Foundation and other contributors <https://openjsf.org/>
+
+Based on Underscore.js, copyright Jeremy Ashkenas,
+DocumentCloud and Investigative Reporters & Editors <http://underscorejs.org/>
+
+This software consists of voluntary contributions made by many
+individuals. For exact contribution history, see the revision history
+available at https://github.com/lodash/lodash
+
+The following license applies to all parts of this software except as
+documented below:
+
+====
+
+Permission is hereby granted, free of charge, to any person obtaining
+a copy of this software and associated documentation files (the
+"Software"), to deal in the Software without restriction, including
+without limitation the rights to use, copy, modify, merge, publish,
+distribute, sublicense, and/or sell copies of the Software, and to
+permit persons to whom the Software is furnished to do so, subject to
+the following conditions:
+
+The above copyright notice and this permission notice shall be
+included in all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
+EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
+MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
+NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
+WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
+
+====
+
+Copyright and related rights for sample code are waived via CC0. Sample
+code is defined as all source code displayed within the prose of the
+documentation.
+
+CC0: http://creativecommons.org/publicdomain/zero/1.0/
+
+====
+
+Files located in the node_modules and vendor directories are externally
+maintained libraries used by this software which have their own
+licenses; we recommend you read them, as their terms may differ from the
+terms above.
+
+
+regenerator-runtime
MIT
+MIT License
-tslib@1.9.3
+Copyright (c) 2014-present, Facebook, Inc.
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+
+roboto-fontface
Apache-2.0
-Apache License
+ Apache License
+ Version 2.0, January 2004
+ http://www.apache.org/licenses/
-Version 2.0, January 2004
+ TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-http://www.apache.org/licenses/
+ 1. Definitions.
-TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+ "License" shall mean the terms and conditions for use, reproduction,
+ and distribution as defined by Sections 1 through 9 of this document.
-1. Definitions.
+ "Licensor" shall mean the copyright owner or entity authorized by
+ the copyright owner that is granting the License.
-"License" shall mean the terms and conditions for use, reproduction, and distribution as defined by Sections 1 through 9 of this document.
+ "Legal Entity" shall mean the union of the acting entity and all
+ other entities that control, are controlled by, or are under common
+ control with that entity. For the purposes of this definition,
+ "control" means (i) the power, direct or indirect, to cause the
+ direction or management of such entity, whether by contract or
+ otherwise, or (ii) ownership of fifty percent (50%) or more of the
+ outstanding shares, or (iii) beneficial ownership of such entity.
-"Licensor" shall mean the copyright owner or entity authorized by the copyright owner that is granting the License.
+ "You" (or "Your") shall mean an individual or Legal Entity
+ exercising permissions granted by this License.
-"Legal Entity" shall mean the union of the acting entity and all other entities that control, are controlled by, or are under common control with that entity. For the purposes of this definition, "control" means (i) the power, direct or indirect, to cause the direction or management of such entity, whether by contract or otherwise, or (ii) ownership of fifty percent (50%) or more of the outstanding shares, or (iii) beneficial ownership of such entity.
+ "Source" form shall mean the preferred form for making modifications,
+ including but not limited to software source code, documentation
+ source, and configuration files.
-"You" (or "Your") shall mean an individual or Legal Entity exercising permissions granted by this License.
+ "Object" form shall mean any form resulting from mechanical
+ transformation or translation of a Source form, including but
+ not limited to compiled object code, generated documentation,
+ and conversions to other media types.
-"Source" form shall mean the preferred form for making modifications, including but not limited to software source code, documentation source, and configuration files.
+ "Work" shall mean the work of authorship, whether in Source or
+ Object form, made available under the License, as indicated by a
+ copyright notice that is included in or attached to the work
+ (an example is provided in the Appendix below).
-"Object" form shall mean any form resulting from mechanical transformation or translation of a Source form, including but not limited to compiled object code, generated documentation, and conversions to other media types.
+ "Derivative Works" shall mean any work, whether in Source or Object
+ form, that is based on (or derived from) the Work and for which the
+ editorial revisions, annotations, elaborations, or other modifications
+ represent, as a whole, an original work of authorship. For the purposes
+ of this License, Derivative Works shall not include works that remain
+ separable from, or merely link (or bind by name) to the interfaces of,
+ the Work and Derivative Works thereof.
-"Work" shall mean the work of authorship, whether in Source or Object form, made available under the License, as indicated by a copyright notice that is included in or attached to the work (an example is provided in the Appendix below).
+ "Contribution" shall mean any work of authorship, including
+ the original version of the Work and any modifications or additions
+ to that Work or Derivative Works thereof, that is intentionally
+ submitted to Licensor for inclusion in the Work by the copyright owner
+ or by an individual or Legal Entity authorized to submit on behalf of
+ the copyright owner. For the purposes of this definition, "submitted"
+ means any form of electronic, verbal, or written communication sent
+ to the Licensor or its representatives, including but not limited to
+ communication on electronic mailing lists, source code control systems,
+ and issue tracking systems that are managed by, or on behalf of, the
+ Licensor for the purpose of discussing and improving the Work, but
+ excluding communication that is conspicuously marked or otherwise
+ designated in writing by the copyright owner as "Not a Contribution."
-"Derivative Works" shall mean any work, whether in Source or Object form, that is based on (or derived from) the Work and for which the editorial revisions, annotations, elaborations, or other modifications represent, as a whole, an original work of authorship. For the purposes of this License, Derivative Works shall not include works that remain separable from, or merely link (or bind by name) to the interfaces of, the Work and Derivative Works thereof.
+ "Contributor" shall mean Licensor and any individual or Legal Entity
+ on behalf of whom a Contribution has been received by Licensor and
+ subsequently incorporated within the Work.
-"Contribution" shall mean any work of authorship, including the original version of the Work and any modifications or additions to that Work or Derivative Works thereof, that is intentionally submitted to Licensor for inclusion in the Work by the copyright owner or by an individual or Legal Entity authorized to submit on behalf of the copyright owner. For the purposes of this definition, "submitted" means any form of electronic, verbal, or written communication sent to the Licensor or its representatives, including but not limited to communication on electronic mailing lists, source code control systems, and issue tracking systems that are managed by, or on behalf of, the Licensor for the purpose of discussing and improving the Work, but excluding communication that is conspicuously marked or otherwise designated in writing by the copyright owner as "Not a Contribution."
+ 2. Grant of Copyright License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ copyright license to reproduce, prepare Derivative Works of,
+ publicly display, publicly perform, sublicense, and distribute the
+ Work and such Derivative Works in Source or Object form.
-"Contributor" shall mean Licensor and any individual or Legal Entity on behalf of whom a Contribution has been received by Licensor and subsequently incorporated within the Work.
+ 3. Grant of Patent License. Subject to the terms and conditions of
+ this License, each Contributor hereby grants to You a perpetual,
+ worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+ (except as stated in this section) patent license to make, have made,
+ use, offer to sell, sell, import, and otherwise transfer the Work,
+ where such license applies only to those patent claims licensable
+ by such Contributor that are necessarily infringed by their
+ Contribution(s) alone or by combination of their Contribution(s)
+ with the Work to which such Contribution(s) was submitted. If You
+ institute patent litigation against any entity (including a
+ cross-claim or counterclaim in a lawsuit) alleging that the Work
+ or a Contribution incorporated within the Work constitutes direct
+ or contributory patent infringement, then any patent licenses
+ granted to You under this License for that Work shall terminate
+ as of the date such litigation is filed.
-2. Grant of Copyright License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable copyright license to reproduce, prepare Derivative Works of, publicly display, publicly perform, sublicense, and distribute the Work and such Derivative Works in Source or Object form.
+ 4. Redistribution. You may reproduce and distribute copies of the
+ Work or Derivative Works thereof in any medium, with or without
+ modifications, and in Source or Object form, provided that You
+ meet the following conditions:
-3. Grant of Patent License. Subject to the terms and conditions of this License, each Contributor hereby grants to You a perpetual, worldwide, non-exclusive, no-charge, royalty-free, irrevocable (except as stated in this section) patent license to make, have made, use, offer to sell, sell, import, and otherwise transfer the Work, where such license applies only to those patent claims licensable by such Contributor that are necessarily infringed by their Contribution(s) alone or by combination of their Contribution(s) with the Work to which such Contribution(s) was submitted. If You institute patent litigation against any entity (including a cross-claim or counterclaim in a lawsuit) alleging that the Work or a Contribution incorporated within the Work constitutes direct or contributory patent infringement, then any patent licenses granted to You under this License for that Work shall terminate as of the date such litigation is filed.
+ (a) You must give any other recipients of the Work or
+ Derivative Works a copy of this License; and
-4. Redistribution. You may reproduce and distribute copies of the Work or Derivative Works thereof in any medium, with or without modifications, and in Source or Object form, provided that You meet the following conditions:
+ (b) You must cause any modified files to carry prominent notices
+ stating that You changed the files; and
-You must give any other recipients of the Work or Derivative Works a copy of this License; and
+ (c) You must retain, in the Source form of any Derivative Works
+ that You distribute, all copyright, patent, trademark, and
+ attribution notices from the Source form of the Work,
+ excluding those notices that do not pertain to any part of
+ the Derivative Works; and
-You must cause any modified files to carry prominent notices stating that You changed the files; and
+ (d) If the Work includes a "NOTICE" text file as part of its
+ distribution, then any Derivative Works that You distribute must
+ include a readable copy of the attribution notices contained
+ within such NOTICE file, excluding those notices that do not
+ pertain to any part of the Derivative Works, in at least one
+ of the following places: within a NOTICE text file distributed
+ as part of the Derivative Works; within the Source form or
+ documentation, if provided along with the Derivative Works; or,
+ within a display generated by the Derivative Works, if and
+ wherever such third-party notices normally appear. The contents
+ of the NOTICE file are for informational purposes only and
+ do not modify the License. You may add Your own attribution
+ notices within Derivative Works that You distribute, alongside
+ or as an addendum to the NOTICE text from the Work, provided
+ that such additional attribution notices cannot be construed
+ as modifying the License.
-You must retain, in the Source form of any Derivative Works that You distribute, all copyright, patent, trademark, and attribution notices from the Source form of the Work, excluding those notices that do not pertain to any part of the Derivative Works; and
+ You may add Your own copyright statement to Your modifications and
+ may provide additional or different license terms and conditions
+ for use, reproduction, or distribution of Your modifications, or
+ for any such Derivative Works as a whole, provided Your use,
+ reproduction, and distribution of the Work otherwise complies with
+ the conditions stated in this License.
-If the Work includes a "NOTICE" text file as part of its distribution, then any Derivative Works that You distribute must include a readable copy of the attribution notices contained within such NOTICE file, excluding those notices that do not pertain to any part of the Derivative Works, in at least one of the following places: within a NOTICE text file distributed as part of the Derivative Works; within the Source form or documentation, if provided along with the Derivative Works; or, within a display generated by the Derivative Works, if and wherever such third-party notices normally appear. The contents of the NOTICE file are for informational purposes only and do not modify the License. You may add Your own attribution notices within Derivative Works that You distribute, alongside or as an addendum to the NOTICE text from the Work, provided that such additional attribution notices cannot be construed as modifying the License. You may add Your own copyright statement to Your modifications and may provide additional or different license terms and conditions for use, reproduction, or distribution of Your modifications, or for any such Derivative Works as a whole, provided Your use, reproduction, and distribution of the Work otherwise complies with the conditions stated in this License.
+ 5. Submission of Contributions. Unless You explicitly state otherwise,
+ any Contribution intentionally submitted for inclusion in the Work
+ by You to the Licensor shall be under the terms and conditions of
+ this License, without any additional terms or conditions.
+ Notwithstanding the above, nothing herein shall supersede or modify
+ the terms of any separate license agreement you may have executed
+ with Licensor regarding such Contributions.
-5. Submission of Contributions. Unless You explicitly state otherwise, any Contribution intentionally submitted for inclusion in the Work by You to the Licensor shall be under the terms and conditions of this License, without any additional terms or conditions. Notwithstanding the above, nothing herein shall supersede or modify the terms of any separate license agreement you may have executed with Licensor regarding such Contributions.
+ 6. Trademarks. This License does not grant permission to use the trade
+ names, trademarks, service marks, or product names of the Licensor,
+ except as required for reasonable and customary use in describing the
+ origin of the Work and reproducing the content of the NOTICE file.
-6. Trademarks. This License does not grant permission to use the trade names, trademarks, service marks, or product names of the Licensor, except as required for reasonable and customary use in describing the origin of the Work and reproducing the content of the NOTICE file.
+ 7. Disclaimer of Warranty. Unless required by applicable law or
+ agreed to in writing, Licensor provides the Work (and each
+ Contributor provides its Contributions) on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+ implied, including, without limitation, any warranties or conditions
+ of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+ PARTICULAR PURPOSE. You are solely responsible for determining the
+ appropriateness of using or redistributing the Work and assume any
+ risks associated with Your exercise of permissions under this License.
-7. Disclaimer of Warranty. Unless required by applicable law or agreed to in writing, Licensor provides the Work (and each Contributor provides its Contributions) on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied, including, without limitation, any warranties or conditions of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A PARTICULAR PURPOSE. You are solely responsible for determining the appropriateness of using or redistributing the Work and assume any risks associated with Your exercise of permissions under this License.
+ 8. Limitation of Liability. In no event and under no legal theory,
+ whether in tort (including negligence), contract, or otherwise,
+ unless required by applicable law (such as deliberate and grossly
+ negligent acts) or agreed to in writing, shall any Contributor be
+ liable to You for damages, including any direct, indirect, special,
+ incidental, or consequential damages of any character arising as a
+ result of this License or out of the use or inability to use the
+ Work (including but not limited to damages for loss of goodwill,
+ work stoppage, computer failure or malfunction, or any and all
+ other commercial damages or losses), even if such Contributor
+ has been advised of the possibility of such damages.
-8. Limitation of Liability. In no event and under no legal theory, whether in tort (including negligence), contract, or otherwise, unless required by applicable law (such as deliberate and grossly negligent acts) or agreed to in writing, shall any Contributor be liable to You for damages, including any direct, indirect, special, incidental, or consequential damages of any character arising as a result of this License or out of the use or inability to use the Work (including but not limited to damages for loss of goodwill, work stoppage, computer failure or malfunction, or any and all other commercial damages or losses), even if such Contributor has been advised of the possibility of such damages.
+ 9. Accepting Warranty or Additional Liability. While redistributing
+ the Work or Derivative Works thereof, You may choose to offer,
+ and charge a fee for, acceptance of support, warranty, indemnity,
+ or other liability obligations and/or rights consistent with this
+ License. However, in accepting such obligations, You may act only
+ on Your own behalf and on Your sole responsibility, not on behalf
+ of any other Contributor, and only if You agree to indemnify,
+ defend, and hold each Contributor harmless for any liability
+ incurred by, or claims asserted against, such Contributor by reason
+ of your accepting any such warranty or additional liability.
-9. Accepting Warranty or Additional Liability. While redistributing the Work or Derivative Works thereof, You may choose to offer, and charge a fee for, acceptance of support, warranty, indemnity, or other liability obligations and/or rights consistent with this License. However, in accepting such obligations, You may act only on Your own behalf and on Your sole responsibility, not on behalf of any other Contributor, and only if You agree to indemnify, defend, and hold each Contributor harmless for any liability incurred by, or claims asserted against, such Contributor by reason of your accepting any such warranty or additional liability.
+ END OF TERMS AND CONDITIONS
-END OF TERMS AND CONDITIONS
+ APPENDIX: How to apply the Apache License to your work.
-rxjs@6.2.1
+ To apply the Apache License to your work, attach the following
+ boilerplate notice, with the fields enclosed by brackets "[]"
+ replaced with your own identifying information. (Don't include
+ the brackets!) The text should be enclosed in the appropriate
+ comment syntax for the file format. We also recommend that a
+ file or class name and description of purpose be included on the
+ same "printed page" as the copyright notice for easier
+ identification within third-party archives.
+
+ Copyright 2013 Christian Hoffmeister
+
+ 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.
+
+
+rxjs
Apache-2.0
-Apache License
+ Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
@@ -357,26 +1466,74 @@
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.
+
-@ngrx/store-devtools@6.0.1
+
+transformation-matrix
MIT
+MIT License
+
+Copyright (c) 2017 https://github.com/chrvadala
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
+
+
+tslib
+0BSD
+Copyright (c) Microsoft Corporation.
+
+Permission to use, copy, modify, and/or distribute this software for any
+purpose with or without fee is hereby granted.
+
+THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH
+REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY
+AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, DIRECT,
+INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM
+LOSS OF USE, DATA OR PROFITS, WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR
+OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE USE OR
+PERFORMANCE OF THIS SOFTWARE.
+
+webcola
MIT
+The MIT License (MIT)
-file-saver@1.3.8
-MIT
-The MIT License
+Copyright (c) 2013 Tim Dwyer
-Copyright © 2016 [Eli Grey][1].
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
-Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions:
+The above copyright notice and this permission notice shall be included in all
+copies or substantial portions of the Software.
-The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software.
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
+SOFTWARE.
-THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-
- [1]: http://eligrey.com
-
-webpack@4.8.3
+webpack
MIT
Copyright JS Foundation and other contributors
@@ -399,6 +1556,27 @@
TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE
SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
-@ngrx/store@6.0.1
+
+zone.js
MIT
-MIT
\ No newline at end of file
+The MIT License
+
+Copyright (c) 2010-2020 Google LLC. http://angular.io/license
+
+Permission is hereby granted, free of charge, to any person obtaining a copy
+of this software and associated documentation files (the "Software"), to deal
+in the Software without restriction, including without limitation the rights
+to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+copies of the Software, and to permit persons to whom the Software is
+furnished to do so, subject to the following conditions:
+
+The above copyright notice and this permission notice shall be included in
+all copies or substantial portions of the Software.
+
+THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+THE SOFTWARE.
diff --git a/hyracks-fullstack/algebricks/algebricks-common/pom.xml b/hyracks-fullstack/algebricks/algebricks-common/pom.xml
index 9b357a2..d288ad8 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-common/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index dbb9363..619b67c 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index f58a7f6..425b6b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index c6b7843..8540d0b 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -108,6 +109,10 @@
* Job generation context.
* @param spec
* Target job specification.
+ * @param secondaryKeysPipelines
+ * Nested plans to extract secondary keys.
+ * @param pipelineTopSchema
+ * Schema of the primary pipeline for secondary keys.
* @return
* A Hyracks IOperatorDescriptor and its partition constraint.
* @throws AlgebricksException
@@ -116,7 +121,9 @@
IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException;
+ JobGenContext context, JobSpecification spec, boolean bulkload,
+ List<List<AlgebricksPipeline>> secondaryKeysPipelines, IOperatorSchema pipelineTopSchema)
+ throws AlgebricksException;
/**
* Creates the delete runtime of IndexInsertDeletePOperator, which models
@@ -147,6 +154,10 @@
* Job generation context.
* @param spec
* Target job specification.
+ * @param secondaryKeysPipelines
+ * Nested plan to extract secondary keys.
+ * @param pipelineTopSchema
+ * Schema of the primary pipeline for secondary keys.
* @return
* A Hyracks IOperatorDescriptor and its partition constraint.
* @throws AlgebricksException
@@ -155,7 +166,8 @@
IDataSourceIndex<I, S> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
List<LogicalVariable> additionalNonKeyFields, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
- JobGenContext context, JobSpecification spec) throws AlgebricksException;
+ JobGenContext context, JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines,
+ IOperatorSchema pipelineTopSchema) throws AlgebricksException;
/**
* Creates the TokenizeOperator for IndexInsertDeletePOperator, which tokenizes
@@ -209,7 +221,7 @@
List<LogicalVariable> additionalFilteringKeys, ILogicalExpression filterExpr,
LogicalVariable upsertIndicatorVar, List<LogicalVariable> prevSecondaryKeys,
LogicalVariable prevAdditionalFilteringKeys, RecordDescriptor inputDesc, JobGenContext context,
- JobSpecification spec) throws AlgebricksException;
+ JobSpecification spec, List<List<AlgebricksPipeline>> secondaryKeysPipelines) throws AlgebricksException;
public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
index c84db0e..7b6ed26 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteUpsertOperator.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
@@ -34,15 +35,49 @@
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
-public class IndexInsertDeleteUpsertOperator extends AbstractLogicalOperator {
+/**
+ * Logical operator for handling secondary index maintenance / loading.
+ * <p>
+ *
+ * In both cases (whether the index is on an atomic field or an array field):
+ * <p>
+ * Primary keys will be given in {@link #primaryKeyExprs}. {@link #operation} specifies the type of index maintenance to
+ * perform. In the case of bulk-loading, {@link #operation} will be INSERT and the {@link #bulkload} flag will be
+ * raised. {@link #additionalFilteringExpressions} and {@link #numberOfAdditionalNonFilteringFields} refers to the
+ * additionalFilteringExpressions, numberOfAdditionalNonFilteringFields found in the corresponding primary index
+ * {@link InsertDeleteUpsertOperator} (i.e. to specify LSM filters). {@link #upsertIndicatorExpr} also originates from
+ * {@link InsertDeleteUpsertOperator}, and is only set when the operation is of kind UPSERT.
+ * <p>
+ *
+ * If the SIDX is on an atomic field <b>or</b> on an array field w/ a bulk-load operation:
+ * <p>
+ * We specify secondary key information in {@link #secondaryKeyExprs}. If we may encounter nullable keys, then we
+ * specify a {@link #filterExpr} to be evaluated inside the runtime. If the operation is of kind UPSERT, then we must
+ * also specify previous secondary key information in {@link #prevSecondaryKeyExprs}. If
+ * {@link #additionalFilteringExpressions} has been set, then {@link #prevAdditionalFilteringExpression} should also be
+ * set.
+ *
+ * <p>
+ * If the SIDX is on an array field <b>and</b> we are not performing a bulk-load operation:
+ * <p>
+ * We <b>do not</b> specify secondary key information in {@link #secondaryKeyExprs} (this is null). Instead, we specify
+ * how to extract secondary keys using {@link #nestedPlans}. If we may encounter nullable keys, then we <b>do not</b>
+ * specify a {@link #filterExpr} (this is null). Instead, this filter must be attached to the top of the nested plan
+ * itself. If the operation is not of type UPSERT, then we must only have one nested plan. Otherwise, the second nested
+ * plan must specify how to extract secondary keys from the previous record. {@link #prevSecondaryKeyExprs} and
+ * {@link #prevAdditionalFilteringExpression} will always be null here, even if the operation is UPSERT.
+ *
+ */
+public class IndexInsertDeleteUpsertOperator extends AbstractOperatorWithNestedPlans {
private final IDataSourceIndex<?, ?> dataSourceIndex;
private final List<Mutable<ILogicalExpression>> primaryKeyExprs;
// In the bulk-load case on ngram or keyword index,
// it contains [token, number of token] or [token].
+ // In the non bulk-load array-index case, it contains nothing.
// Otherwise, it contains secondary key information.
- private final List<Mutable<ILogicalExpression>> secondaryKeyExprs;
- private final Mutable<ILogicalExpression> filterExpr;
+ private List<Mutable<ILogicalExpression>> secondaryKeyExprs;
+ private Mutable<ILogicalExpression> filterExpr;
private final Kind operation;
private final boolean bulkload;
private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
@@ -66,12 +101,6 @@
}
@Override
- public void recomputeSchema() throws AlgebricksException {
- schema = new ArrayList<LogicalVariable>();
- schema.addAll(inputs.get(0).getValue().getSchema());
- }
-
- @Override
public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
boolean b = false;
// Primary
@@ -125,8 +154,37 @@
}
@Override
- public boolean isMap() {
- return false;
+ public void getUsedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ for (Mutable<ILogicalExpression> e : getPrimaryKeyExpressions()) {
+ e.getValue().getUsedVariables(vars);
+ }
+ for (Mutable<ILogicalExpression> e : getSecondaryKeyExpressions()) {
+ e.getValue().getUsedVariables(vars);
+ }
+ if (getFilterExpression() != null) {
+ getFilterExpression().getValue().getUsedVariables(vars);
+ }
+ if (getAdditionalFilteringExpressions() != null) {
+ for (Mutable<ILogicalExpression> e : getAdditionalFilteringExpressions()) {
+ e.getValue().getUsedVariables(vars);
+ }
+ }
+ if (getPrevAdditionalFilteringExpression() != null) {
+ getPrevAdditionalFilteringExpression().getValue().getUsedVariables(vars);
+ }
+ if (getPrevSecondaryKeyExprs() != null) {
+ for (Mutable<ILogicalExpression> e : getPrevSecondaryKeyExprs()) {
+ e.getValue().getUsedVariables(vars);
+ }
+ }
+ if (getUpsertIndicatorExpr() != null) {
+ getUpsertIndicatorExpr().getValue().getUsedVariables(vars);
+ }
+ }
+
+ @Override
+ public void getProducedVariablesExceptNestedPlans(Collection<LogicalVariable> vars) {
+ // Do nothing (no variables are produced here).
}
@Override
@@ -144,6 +202,12 @@
return createPropagatingAllInputsTypeEnvironment(ctx);
}
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<>();
+ schema.addAll(inputs.get(0).getValue().getSchema());
+ }
+
public List<Mutable<ILogicalExpression>> getPrimaryKeyExpressions() {
return primaryKeyExprs;
}
@@ -160,10 +224,18 @@
return secondaryKeyExprs;
}
+ public void setSecondaryKeyExprs(List<Mutable<ILogicalExpression>> secondaryKeyExprs) {
+ this.secondaryKeyExprs = secondaryKeyExprs;
+ }
+
public Mutable<ILogicalExpression> getFilterExpression() {
return filterExpr;
}
+ public void setFilterExpression(Mutable<ILogicalExpression> filterExpr) {
+ this.filterExpr = filterExpr;
+ }
+
public Kind getOperation() {
return operation;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index ccdbd1f..f3717c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -556,6 +556,17 @@
public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, IOptimizationContext ctx)
throws AlgebricksException {
propagateFDsAndEquivClasses(op, ctx);
+ if (!op.getNestedPlans().isEmpty()) {
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses = ctx.getEquivalenceClassMap(op);
+ List<FunctionalDependency> functionalDependencies = ctx.getFDList(op);
+ for (ILogicalPlan p : op.getNestedPlans()) {
+ for (Mutable<ILogicalOperator> r : p.getRoots()) {
+ ILogicalOperator op2 = r.getValue();
+ equivalenceClasses.putAll(getOrComputeEqClasses(op2, ctx));
+ functionalDependencies.addAll(getOrComputeFDs(op2, ctx));
+ }
+ }
+ }
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 09d0c14..a9f9626 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -601,11 +601,30 @@
return Boolean.FALSE;
}
IndexInsertDeleteUpsertOperator insertOpArg = (IndexInsertDeleteUpsertOperator) copyAndSubstituteVar(op, arg);
- boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema());
- if (!op.getDataSourceIndex().equals(insertOpArg.getDataSourceIndex())) {
- isomorphic = false;
+ if (!VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema())) {
+ return Boolean.FALSE;
}
- return isomorphic;
+
+ // Verify that our index expressions and variables are equal.
+ if (!Objects.equals(op.getDataSourceIndex(), insertOpArg.getDataSourceIndex())
+ || !Objects.equals(op.getPrimaryKeyExpressions(), insertOpArg.getPrimaryKeyExpressions())
+ || !Objects.equals(op.getSecondaryKeyExpressions(), insertOpArg.getSecondaryKeyExpressions())
+ || !Objects.equals(op.getFilterExpression(), insertOpArg.getFilterExpression())
+ || !Objects.equals(op.getOperation(), insertOpArg.getOperation())
+ || (op.isBulkload() != insertOpArg.isBulkload())
+ || !Objects.equals(op.getAdditionalFilteringExpressions(),
+ insertOpArg.getAdditionalFilteringExpressions())
+ || !Objects.equals(op.getPrevSecondaryKeyExprs(), insertOpArg.getPrevSecondaryKeyExprs())
+ || !Objects.equals(op.getPrevAdditionalFilteringExpression(),
+ insertOpArg.getPrevAdditionalFilteringExpression())
+ || !Objects.equals(op.getUpsertIndicatorExpr(), insertOpArg.getUpsertIndicatorExpr())
+ || (op.getNumberOfAdditionalNonFilteringFields() != insertOpArg
+ .getNumberOfAdditionalNonFilteringFields())) {
+ return Boolean.FALSE;
+ }
+
+ // Check our nested plans as well.
+ return compareSubplans(op.getNestedPlans(), insertOpArg.getNestedPlans());
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 5cf0b5f..8ca2b83 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -303,7 +303,8 @@
@Override
public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, ILogicalOperator arg)
throws AlgebricksException {
- mapVariablesStandard(op, arg);
+ mapChildren(op, arg);
+ mapVariablesInNestedPlans(op, arg);
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 46999b0..c06ad5c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -330,6 +330,9 @@
op.getDataSourceIndex(), newPrimaryKeyExpressions, newSecondaryKeyExpressions, newFilterExpression,
op.getOperation(), op.isBulkload(), op.getNumberOfAdditionalNonFilteringFields());
indexInsertDeleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
+ for (ILogicalPlan plan : op.getNestedPlans()) {
+ indexInsertDeleteOp.getNestedPlans().add(OperatorManipulationUtil.deepCopy(plan, indexInsertDeleteOp));
+ }
return indexInsertDeleteOp;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index bd90729..44bb7e2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -309,6 +309,7 @@
@Override
public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
throws AlgebricksException {
+ // Variables in our nested plan are not propagated.
standardLayout(op);
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index a2107e5..6e8b425 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -456,6 +456,9 @@
substUsedVariablesInExpr(op.getUpsertIndicatorExpr(), pair.first, pair.second);
substUsedVariablesInExpr(op.getPrevSecondaryKeyExprs(), pair.first, pair.second);
substUsedVariablesInExpr(op.getPrevAdditionalFilteringExpression(), pair.first, pair.second);
+ if (!op.getNestedPlans().isEmpty()) {
+ substInNestedPlans(op, pair.first, pair.second);
+ }
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 65e9023..23fe3b2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -405,7 +405,8 @@
}
@Override
- public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg) {
+ public Void visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void arg)
+ throws AlgebricksException {
for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
e.getValue().getUsedVariables(usedVariables);
}
@@ -431,6 +432,7 @@
if (op.getUpsertIndicatorExpr() != null) {
op.getUpsertIndicatorExpr().getValue().getUsedVariables(usedVariables);
}
+ visitNestedPlans(op);
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
index 438d9ee..4bc7502 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPhysicalOperator.java
@@ -137,7 +137,7 @@
protected AlgebricksPipeline[] compileSubplans(IOperatorSchema outerPlanSchema,
AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context)
throws AlgebricksException {
- List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(outerPlanSchema, npOp, opSchema, context);
+ List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(outerPlanSchema, npOp, opSchema, context, true);
int n = subplans.size();
AlgebricksPipeline[] result = new AlgebricksPipeline[n];
for (int i = 0; i < n; i++) {
@@ -153,26 +153,28 @@
}
protected List<List<AlgebricksPipeline>> compileSubplansImpl(IOperatorSchema outerPlanSchema,
- AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context)
- throws AlgebricksException {
+ AbstractOperatorWithNestedPlans npOp, IOperatorSchema opSchema, JobGenContext context,
+ boolean withProjection) throws AlgebricksException {
List<List<AlgebricksPipeline>> subplans = new ArrayList<>(npOp.getNestedPlans().size());
PlanCompiler pc = new PlanCompiler(context);
for (ILogicalPlan p : npOp.getNestedPlans()) {
- subplans.add(buildPipelineWithProjection(p, outerPlanSchema, opSchema, pc));
+ subplans.add(buildPipeline(p, outerPlanSchema, opSchema, pc, withProjection));
}
return subplans;
}
- private List<AlgebricksPipeline> buildPipelineWithProjection(ILogicalPlan p, IOperatorSchema outerPlanSchema,
- IOperatorSchema opSchema, PlanCompiler pc) throws AlgebricksException {
+ private List<AlgebricksPipeline> buildPipeline(ILogicalPlan p, IOperatorSchema outerPlanSchema,
+ IOperatorSchema opSchema, PlanCompiler pc, boolean withProjection) throws AlgebricksException {
if (p.getRoots().size() > 1) {
throw new NotImplementedException("Nested plans with several roots are not supported.");
}
JobSpecification nestedJob = pc.compileNestedPlan(p, outerPlanSchema);
- ILogicalOperator topOpInSubplan = p.getRoots().get(0).getValue();
- JobGenContext context = pc.getContext();
- IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
- opSchema.addAllVariables(topOpInSubplanScm);
+ if (withProjection) {
+ ILogicalOperator topOpInSubplan = p.getRoots().get(0).getValue();
+ JobGenContext context = pc.getContext();
+ IOperatorSchema topOpInSubplanScm = context.getSchema(topOpInSubplan);
+ opSchema.addAllVariables(topOpInSubplanScm);
+ }
Map<OperatorDescriptorId, IOperatorDescriptor> opMap = nestedJob.getOperatorMap();
List<? extends IOperatorDescriptor> metaOps = nestedJob.getMetaOps();
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index 6512700..7d70117 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -134,7 +134,7 @@
context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints =
mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
- secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true);
+ secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, true, null, null);
IOperatorDescriptor opDesc = runtimeAndConstraints.first;
opDesc.setSourceLocation(indexInsertDeleteOp.getSourceLocation());
builder.contributeHyracksOperator(indexInsertDeleteOp, opDesc);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
index 228ca52..2d9dabe 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
@@ -43,8 +43,11 @@
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.core.jobgen.impl.PlanCompiler;
+import org.apache.hyracks.algebricks.runtime.base.AlgebricksPipeline;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.job.JobSpecification;
public class IndexInsertDeleteUpsertPOperator extends AbstractPhysicalOperator {
@@ -113,8 +116,27 @@
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
IndexInsertDeleteUpsertOperator insertDeleteUpsertOp = (IndexInsertDeleteUpsertOperator) op;
- IMetadataProvider mp = context.getMetadataProvider();
+ // Compile our nested plans if any exist.
+ int numberOfNestedPlans = insertDeleteUpsertOp.getNestedPlans().size();
+ List<List<AlgebricksPipeline>> secondaryKeyPipelines = null;
+ IOperatorSchema pipelineTopSchema = null;
+ if ((numberOfNestedPlans > 1 && !insertDeleteUpsertOp.getOperation().equals(Kind.UPSERT))
+ || numberOfNestedPlans > 2) {
+ throw AlgebricksException.create(ErrorCode.ILLEGAL_STATE, op.getSourceLocation(),
+ "Using multiple nested plans to extract secondary keys is not supported.");
+ }
+ if (numberOfNestedPlans > 0) {
+ secondaryKeyPipelines =
+ compileSubplansImpl(inputSchemas[0], insertDeleteUpsertOp, propagatedSchema, context, false);
+
+ // Get the schema of our nested plan root.
+ ILogicalOperator topOpInSubplan = insertDeleteUpsertOp.getNestedPlans().get(0).getRoots().get(0).getValue();
+ PlanCompiler pc = new PlanCompiler(context);
+ pipelineTopSchema = pc.getContext().getSchema(topOpInSubplan);
+ }
+
+ IMetadataProvider mp = context.getMetadataProvider();
JobSpecification spec = builder.getJobSpec();
RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
@@ -124,19 +146,19 @@
Kind operation = insertDeleteUpsertOp.getOperation();
switch (operation) {
case INSERT:
- runtimeAndConstraints =
- mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
- secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec, false);
+ runtimeAndConstraints = mp.getIndexInsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+ typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context,
+ spec, false, secondaryKeyPipelines, pipelineTopSchema);
break;
case DELETE:
- runtimeAndConstraints =
- mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas, typeEnv, primaryKeys,
- secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context, spec);
+ runtimeAndConstraints = mp.getIndexDeleteRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
+ typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, inputDesc, context,
+ spec, secondaryKeyPipelines, pipelineTopSchema);
break;
case UPSERT:
runtimeAndConstraints = mp.getIndexUpsertRuntime(dataSourceIndex, propagatedSchema, inputSchemas,
typeEnv, primaryKeys, secondaryKeys, additionalFilteringKeys, filterExpr, upsertIndicatorVar,
- prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec);
+ prevSecondaryKeys, prevAdditionalFilteringKey, inputDesc, context, spec, secondaryKeyPipelines);
break;
default:
throw new AlgebricksException("Unsupported Operation " + operation);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
index 5ec6d0a..898fbe8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -89,7 +89,8 @@
if (subplan.getNestedPlans().size() != 1) {
throw new NotImplementedException("Subplan currently works only for one nested plan with one root.");
}
- List<List<AlgebricksPipeline>> subplans = compileSubplansImpl(inputSchemas[0], subplan, opSchema, context);
+ List<List<AlgebricksPipeline>> subplans =
+ compileSubplansImpl(inputSchemas[0], subplan, opSchema, context, true);
assert subplans.size() == 1;
List<AlgebricksPipeline> np = subplans.get(0);
RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index f6e4742..f17fa70 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -461,7 +461,10 @@
buffer.append(" replace:");
pprintExprList(op.getPrevSecondaryKeyExprs(), indent);
buffer.append(" with:");
- pprintExprList(op.getSecondaryKeyExpressions(), indent);
+ }
+ if (!op.getNestedPlans().isEmpty()) {
+ buffer.append("{");
+ printNestedPlans(op, indent);
} else {
pprintExprList(op.getSecondaryKeyExpressions(), indent);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 6ba4c0c..ed5299c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -625,9 +625,15 @@
jsonGenerator.writeObjectFieldStart("from");
if (op.getOperation() == Kind.UPSERT) {
writeArrayFieldOfExpressions("replace", op.getPrevSecondaryKeyExprs(), indent);
- writeArrayFieldOfExpressions("with", op.getSecondaryKeyExpressions(), indent);
- } else {
+ if (op.getNestedPlans().isEmpty()) {
+ writeArrayFieldOfExpressions("with", op.getSecondaryKeyExpressions(), indent);
+ } else {
+ writeNestedPlans(op, indent);
+ }
+ } else if (op.getNestedPlans().isEmpty()) {
writeArrayFieldOfExpressions(EXPRESSIONS_FIELD, op.getSecondaryKeyExpressions(), indent);
+ } else {
+ writeNestedPlans(op, indent);
}
jsonGenerator.writeEndObject();
if (op.isBulkload()) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 817db5a..20632f1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -34,6 +34,7 @@
public static final boolean SUBPLAN_MERGE_DEFAULT = true;
public static final boolean SUBPLAN_NESTEDPUSHDOWN_DEFAULT = true;
public static final boolean MIN_MEMORY_ALLOCATION_DEFAULT = true;
+ public static final boolean ARRAY_INDEX_DEFAULT = false;
public static final int EXTERNAL_SCAN_BUFFER_SIZE =
StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index bfff925..62bf671 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -45,6 +45,7 @@
private static final String SUBPLAN_MERGE = "SUBPLAN_MERGE";
private static final String SUBPLAN_NESTEDPUSHDOWN = "SUBPLAN_NESTEDPUSHDOWN";
private static final String MIN_MEMORY_ALLOCATION = "MIN_MEMORY_ALLOCATION";
+ private static final String ARRAY_INDEX = "ARRAY_INDEX";
private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
private final Properties properties = new Properties();
@@ -227,6 +228,14 @@
setBoolean(MIN_MEMORY_ALLOCATION, value);
}
+ public boolean isArrayIndexEnabled() {
+ return getBoolean(ARRAY_INDEX, AlgebricksConfig.ARRAY_INDEX_DEFAULT);
+ }
+
+ public void setArrayIndexEnabled(boolean arrayIndex) {
+ setBoolean(ARRAY_INDEX, arrayIndex);
+ }
+
public int getExternalScanBufferSize() {
return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
index 67963ce..01f7403 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
@@ -529,6 +529,10 @@
printExprList(op.getPrevSecondaryKeyExprs());
stringBuilder.append(" with:");
printExprList(op.getSecondaryKeyExpressions());
+ }
+ if (!op.getNestedPlans().isEmpty()) {
+ // TODO: Finish this for nested plans.
+ stringBuilder.append("{ a nested plan }");
} else {
printExprList(op.getSecondaryKeyExpressions());
}
diff --git a/hyracks-fullstack/algebricks/algebricks-data/pom.xml b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
index 43d119b..77f9367 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
index 39b1ba6..3b4cc1a 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
index 0138946..586f240 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
index 81f5d08..858fcfa 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/PipelineAssembler.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.EnforceFrameWriter;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobFlag;
@@ -123,4 +124,21 @@
}
return start;
}
+
+ public static IPushRuntime linkPipeline(AlgebricksPipeline pipeline, PipelineAssembler[] pipelineAssemblers,
+ int pipelineAssemblersCount) throws HyracksDataException {
+ IPushRuntimeFactory[] outputRuntimeFactories = pipeline.getOutputRuntimeFactories();
+ if (outputRuntimeFactories == null || outputRuntimeFactories.length != 1) {
+ throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE, "No output runtime factories found.");
+ }
+ IPushRuntimeFactory outRuntimeFactory = outputRuntimeFactories[0];
+ int outputPosition = pipeline.getOutputPositions()[0];
+ for (int i = 0; i < pipelineAssemblersCount; i++) {
+ IPushRuntime[] p = pipelineAssemblers[i].getPushRuntime(outRuntimeFactory);
+ if (p != null) {
+ return p[outputPosition];
+ }
+ }
+ return null;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 11aefc6..349e50f 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -117,7 +117,7 @@
outputRecordDescriptor = SubplanRuntimeFactory.this.outputRecordDesc;
} else {
// secondary pipeline
- IPushRuntime outputPushRuntime = linkSecondaryPipeline(pipeline, pipelineAssemblers, i);
+ IPushRuntime outputPushRuntime = PipelineAssembler.linkPipeline(pipeline, pipelineAssemblers, i);
if (outputPushRuntime == null) {
throw new IllegalStateException("Invalid pipeline");
}
@@ -132,23 +132,6 @@
}
}
- IPushRuntime linkSecondaryPipeline(AlgebricksPipeline pipeline, PipelineAssembler[] pipelineAssemblers,
- int pipelineAssemblersCount) {
- IPushRuntimeFactory[] outputRuntimeFactories = pipeline.getOutputRuntimeFactories();
- if (outputRuntimeFactories == null || outputRuntimeFactories.length != 1) {
- throw new IllegalStateException();
- }
- IPushRuntimeFactory outRuntimeFactory = outputRuntimeFactories[0];
- int outputPosition = pipeline.getOutputPositions()[0];
- for (int i = 0; i < pipelineAssemblersCount; i++) {
- IPushRuntime[] p = pipelineAssemblers[i].getPushRuntime(outRuntimeFactory);
- if (p != null) {
- return p[outputPosition];
- }
- }
- return null;
- }
-
@Override
public void open() throws HyracksDataException {
// writer opened many times?
diff --git a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
index b2b4d50..9f689e3 100644
--- a/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-tests/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>algebricks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/algebricks/pom.xml b/hyracks-fullstack/algebricks/pom.xml
index 1f06db2..c961c0c 100644
--- a/hyracks-fullstack/algebricks/pom.xml
+++ b/hyracks-fullstack/algebricks/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks-fullstack-license/pom.xml b/hyracks-fullstack/hyracks-fullstack-license/pom.xml
index 19df842..50f0a0f 100644
--- a/hyracks-fullstack/hyracks-fullstack-license/pom.xml
+++ b/hyracks-fullstack/hyracks-fullstack-license/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index 2fab94a..1ba086c 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-client/pom.xml b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
index fcb5821..895cee2 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-client/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/pom.xml b/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
index 98c3586..f9a5060 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-comm/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index 0f1be96..57d9c09 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
index 1b5b38d..2a5db06 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index a52930c..5898622 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-control</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
index 26086f0..2048509 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-nc-service/pom.xml
@@ -21,7 +21,7 @@
<parent>
<artifactId>hyracks-control</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/pom.xml
index 6cb4d75..e8fb251 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
index fb143d9..48f3b33 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-data</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
<root.dir>${basedir}/../../..</root.dir>
diff --git a/hyracks-fullstack/hyracks/hyracks-data/pom.xml b/hyracks-fullstack/hyracks/hyracks-data/pom.xml
index caa6e78..0be145a 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-data/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
index a5a474f..f2ab957 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index f51ef86..ea9b704 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-dist/pom.xml b/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
index 90d4f5a..ceeea81 100644
--- a/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dist/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml b/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
index 58260a7..ecb80a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-documentation/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
index 4c36a7f..c5f2c23 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeclient/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
index 3c513d3..126ee54 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreehelper/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
index 4cd94d4..72919ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/btreeserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks.examples</groupId>
<artifactId>btree-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
index 94ecd5b..47525d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/btree-example/pom.xml
@@ -27,7 +27,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 8c543f3..35a6bdd 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
index 65ada11..cd55b48 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-shutdown-test/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
index d295986..ca9d937 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
index 6696566..a03ab9b 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
index 4590c60..ef07b62 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textclient/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
index 8ea4601..dc450a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
index ec4ca99..1a276ca 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/textserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>text-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
index 1aef31b..b7d1754 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-examples</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
index 3d7b534..ec7d353 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchclient/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>tpch-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
index fd49e1a..143d0c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/tpch-example/tpchserver/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>tpch-example</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index d418548..c395145 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index 0c94743..23eb3db 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -22,7 +22,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<artifactId>hyracks-http</artifactId>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
index bcc5be4..29481e8 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
index bb074ac..0d5e4a3 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-maven-plugins</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
index a7907a7..9103d55 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
@@ -21,7 +21,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-maven-plugins</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<packaging>maven-plugin</packaging>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
index 7613ec4..dde3f05 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 5ebfb2e..9929f64 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-server/pom.xml b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
index b8b7c1a..b76e81c 100644
--- a/hyracks-fullstack/hyracks/hyracks-server/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-server/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
index fca0ea1..41bd766 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-bloomfilter/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index 37e9959..745599b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index af7ec3d..e23fa44 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
index 887a48f..711932e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
index bba6cff..78c4cf0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
index 366c2a7..98e4e6d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/pom.xml
@@ -23,7 +23,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
<licenses>
@@ -96,5 +96,14 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+
</dependencies>
</project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index 3ed47e6..4477e65 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -27,6 +27,9 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
public class BinaryTokenizerOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -34,6 +37,7 @@
private static final long serialVersionUID = 1L;
private final IBinaryTokenizerFactory tokenizerFactory;
+ private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
// Field that will be tokenized.
private final int docField;
// operator will append these key fields to each token, e.g., as
@@ -53,10 +57,12 @@
private final IMissingWriterFactory missingWriterFactory;
public BinaryTokenizerOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor recDesc,
- IBinaryTokenizerFactory tokenizerFactory, int docField, int[] keyFields, boolean addNumTokensKey,
- boolean writeKeyFieldsFirst, boolean writeMissing, IMissingWriterFactory missingWriterFactory) {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ int docField, int[] keyFields, boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
+ IMissingWriterFactory missingWriterFactory) {
super(spec, 1, 1);
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.docField = docField;
this.keyFields = keyFields;
this.addNumTokensKey = addNumTokensKey;
@@ -69,9 +75,14 @@
@Override
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+ IFullTextConfigEvaluator fullTextConfigEvaluator =
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
return new BinaryTokenizerOperatorNodePushable(ctx,
- recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), outRecDescs[0],
- tokenizerFactory.createTokenizer(), docField, keyFields, addNumTokensKey, writeKeyFieldsFirst,
- writeMissing, missingWriterFactory);
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), outRecDescs[0], tokenizer,
+ fullTextConfigEvaluator, docField, keyFields, addNumTokensKey, writeKeyFieldsFirst, writeMissing,
+ missingWriterFactory);
}
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
index 3df185a..b9c8c8a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/BinaryTokenizerOperatorNodePushable.java
@@ -36,13 +36,14 @@
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
public class BinaryTokenizerOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
private final IHyracksTaskContext ctx;
- private final IBinaryTokenizer tokenizer;
+ private final IFullTextConfigEvaluator fullTextEvaluator;
private final int docField;
private final int[] keyFields;
private final boolean addNumTokensKey;
@@ -59,11 +60,24 @@
private FrameTupleAppender appender;
public BinaryTokenizerOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc,
- RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, int docField, int[] keyFields,
- boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
+ RecordDescriptor outputRecDesc, IBinaryTokenizer tokenizer, IFullTextConfigEvaluator fullTextEvaluator,
+ int docField, int[] keyFields, boolean addNumTokensKey, boolean writeKeyFieldsFirst, boolean writeMissing,
IMissingWriterFactory missingWriterFactory) {
this.ctx = ctx;
- this.tokenizer = tokenizer;
+ this.fullTextEvaluator = fullTextEvaluator;
+ // Need to use the tokenizer created in the upper-layer when:
+ // 1. The tokenizer is of TokenizerCategory.NGram rather than Word
+ // 2. If the tokenizer is a TokenizerCategory.Word one, then its parameters
+ // (e.g. boolean ignoreTokenCount, boolean sourceHasTypeTag) may be different
+ // from the tokenizer in the default full-text config.
+ //
+ // Note that those parameters might be call-site specific, for example, one string byte array may contains
+ // the ATypeTag.String in it while some doesn't. Even though the tokenizers are both Word tokenizer,
+ // we still need to set different tokenizer here.
+ // The different tokeniers are defined in BinaryTokenizerFactoryProvider.
+ // The big plan is to remove the tokenizer from this interface and use fullTextEvaluator only.
+ this.fullTextEvaluator.setTokenizer(tokenizer);
+
this.docField = docField;
this.keyFields = keyFields;
this.addNumTokensKey = addNumTokensKey;
@@ -91,20 +105,20 @@
for (int i = 0; i < tupleCount; i++) {
tuple.reset(accessor, i);
- short numTokens = 0;
+ int numTokens = 0;
if (!isDocFieldMissing(tuple)) {
- tokenizer.reset(tuple.getFieldData(docField), tuple.getFieldStart(docField),
+ fullTextEvaluator.reset(tuple.getFieldData(docField), tuple.getFieldStart(docField),
tuple.getFieldLength(docField));
if (addNumTokensKey) {
// Get the total number of tokens.
- numTokens = tokenizer.getTokensCount();
+ numTokens = fullTextEvaluator.getTokensCount();
}
// Write token and data into frame by following the order specified
// in the writeKeyFieldsFirst field.
- while (tokenizer.hasNext()) {
- tokenizer.next();
- IToken token = tokenizer.getToken();
+ while (fullTextEvaluator.hasNext()) {
+ fullTextEvaluator.next();
+ IToken token = fullTextEvaluator.getToken();
writeTuple(token, numTokens, i);
}
} else if (writeMissing) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
index 0904c4a..0af1f3f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResource.java
@@ -42,6 +42,8 @@
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResource;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -54,33 +56,50 @@
public class LSMInvertedIndexLocalResource extends LsmResource {
private static final long serialVersionUID = 1L;
+ public static final String TOKEN_TYPE_TRAITS = "tokenTypeTraits";
+ public static final String TOKEN_CMP_FACTORIES = "tokenCmpFactories";
+ public static final String IS_PARTITIONED = "isPartitioned";
+ public static final String FULL_TEXT_CONFIG_EVALUATOR_FACTORY = "fullTextConfigEvaluatorFactory";
private final ITypeTraits[] tokenTypeTraits;
private final IBinaryComparatorFactory[] tokenCmpFactories;
+ // ToDo: totally replace tokenizer with fullTextConfig
+ // Currently, the configuration of tokenizer is complicated and the parameters
+ // (e.g. whether the string contains a ATypeTag.AString at the beginning),
+ // as a first step, we set the tokenizer of the full-text config on-the-fly
+ // See comments in FullTextConfigEvaluator
private final IBinaryTokenizerFactory tokenizerFactory;
+ private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
private final boolean isPartitioned;
private final int[] invertedIndexFields;
private final int[] filterFieldsForNonBulkLoadOps;
private final int[] invertedIndexFieldsForNonBulkLoadOps;
private final double bloomFilterFalsePositiveRate;
- public LSMInvertedIndexLocalResource(String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
+ public LSMInvertedIndexLocalResource(
+ // inherited fields
+ String path, IStorageManager storageManager, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerProvider, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
- Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ Map<String, String> mergePolicyProperties, boolean durable,
+ // new fields
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
+
super(path, storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
opTrackerProvider, ioOpCallbackFactory, pageWriteCallbackFactory, metadataPageManagerFactory,
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
+
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.isPartitioned = isPartitioned;
this.invertedIndexFields = invertedIndexFields;
this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -90,13 +109,15 @@
private LSMInvertedIndexLocalResource(IPersistedResourceRegistry registry, JsonNode json,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, boolean isPartitioned, int[] invertedIndexFields,
- int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
- double bloomFilterFalsePositiveRate) throws HyracksDataException {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
+ int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate)
+ throws HyracksDataException {
super(registry, json);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.isPartitioned = isPartitioned;
this.invertedIndexFields = invertedIndexFields;
this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -116,19 +137,20 @@
pageWriteCallbackFactory.initialize(serviceCtx, this);
if (isPartitioned) {
return InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits,
- cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache,
- file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
+ cmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
+ bufferCache, file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
metadataPageManagerFactory, serviceCtx.getTracer());
} else {
return InvertedIndexUtils.createLSMInvertedIndex(ioManager, virtualBufferCaches, typeTraits, cmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, bufferCache, file.getAbsolutePath(),
- bloomFilterFalsePositiveRate, mergePolicy, opTrackerProvider.getOperationTracker(serviceCtx, this),
- ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits,
- filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable, metadataPageManagerFactory, serviceCtx.getTracer());
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, bufferCache,
+ file.getAbsolutePath(), bloomFilterFalsePositiveRate, mergePolicy,
+ opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
+ pageWriteCallbackFactory, invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
+ metadataPageManagerFactory, serviceCtx.getTracer());
}
}
@@ -140,14 +162,15 @@
for (ITypeTraits tt : tokenTypeTraits) {
tokenTypeTraitsArray.add(tt.toJson(registry));
}
- jsonObject.set("tokenTypeTraits", tokenTypeTraitsArray);
+ jsonObject.set(TOKEN_TYPE_TRAITS, tokenTypeTraitsArray);
final ArrayNode tokenCmpFactoriesArray = OBJECT_MAPPER.createArrayNode();
for (IBinaryComparatorFactory factory : tokenCmpFactories) {
tokenCmpFactoriesArray.add(factory.toJson(registry));
}
- jsonObject.set("tokenCmpFactories", tokenCmpFactoriesArray);
+ jsonObject.set(TOKEN_CMP_FACTORIES, tokenCmpFactoriesArray);
jsonObject.set("tokenizerFactory", tokenizerFactory.toJson(registry));
- jsonObject.put("isPartitioned", isPartitioned);
+ jsonObject.set(FULL_TEXT_CONFIG_EVALUATOR_FACTORY, fullTextConfigEvaluatorFactory.toJson(registry));
+ jsonObject.put(IS_PARTITIONED, isPartitioned);
jsonObject.putPOJO("invertedIndexFields", invertedIndexFields);
jsonObject.putPOJO("filterFieldsForNonBulkLoadOps", filterFieldsForNonBulkLoadOps);
jsonObject.putPOJO("invertedIndexFieldsForNonBulkLoadOps", invertedIndexFieldsForNonBulkLoadOps);
@@ -173,7 +196,17 @@
tokenCmpFactoriesList.toArray(new IBinaryComparatorFactory[0]);
final IBinaryTokenizerFactory tokenizerFactory =
(IBinaryTokenizerFactory) registry.deserialize(json.get("tokenizerFactory"));
- final boolean isPartitioned = json.get("isPartitioned").asBoolean();
+
+ final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
+ // back-compatible: the local resource in an older version of AsterixDB may not contain the newly added fullTextConfigFactory
+ if (json.has(FULL_TEXT_CONFIG_EVALUATOR_FACTORY)) {
+ fullTextConfigEvaluatorFactory = (IFullTextConfigEvaluatorFactory) registry
+ .deserialize(json.get(FULL_TEXT_CONFIG_EVALUATOR_FACTORY));
+ } else {
+ fullTextConfigEvaluatorFactory = FullTextConfigEvaluatorFactory.getDefaultFactory();
+ }
+
+ final boolean isPartitioned = json.get(IS_PARTITIONED).asBoolean();
final int[] invertedIndexFields = OBJECT_MAPPER.convertValue(json.get("invertedIndexFields"), int[].class);
final int[] filterFieldsForNonBulkLoadOps =
OBJECT_MAPPER.convertValue(json.get("filterFieldsForNonBulkLoadOps"), int[].class);
@@ -181,7 +214,7 @@
OBJECT_MAPPER.convertValue(json.get("invertedIndexFieldsForNonBulkLoadOps"), int[].class);
final double bloomFilterFalsePositiveRate = json.get("bloomFilterFalsePositiveRate").asDouble();
return new LSMInvertedIndexLocalResource(registry, json, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps,
- bloomFilterFalsePositiveRate);
+ fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
index c36e8d3..09c9699 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexLocalResourceFactory.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMPageWriteCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCacheProvider;
import org.apache.hyracks.storage.am.lsm.common.dataflow.LsmResourceFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.IResource;
import org.apache.hyracks.storage.common.IStorageManager;
@@ -46,16 +47,20 @@
private final int[] filterFieldsForNonBulkLoadOps;
private final int[] invertedIndexFieldsForNonBulkLoadOps;
private final double bloomFilterFalsePositiveRate;
+ private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
public LSMInvertedIndexLocalResourceFactory(IStorageManager storageManager, ITypeTraits[] typeTraits,
+ // Inherited fields
IBinaryComparatorFactory[] cmpFactories, ITypeTraits[] filterTypeTraits,
IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
ILSMOperationTrackerFactory opTrackerFactory, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
IMetadataPageManagerFactory metadataPageManagerFactory, IVirtualBufferCacheProvider vbcProvider,
ILSMIOOperationSchedulerProvider ioSchedulerProvider, ILSMMergePolicyFactory mergePolicyFactory,
- Map<String, String> mergePolicyProperties, boolean durable, ITypeTraits[] tokenTypeTraits,
- IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
+ Map<String, String> mergePolicyProperties, boolean durable,
+ // New fields
+ ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
boolean isPartitioned, int[] invertedIndexFields, int[] filterFieldsForNonBulkLoadOps,
int[] invertedIndexFieldsForNonBulkLoadOps, double bloomFilterFalsePositiveRate) {
super(storageManager, typeTraits, cmpFactories, filterTypeTraits, filterCmpFactories, filterFields,
@@ -63,7 +68,9 @@
vbcProvider, ioSchedulerProvider, mergePolicyFactory, mergePolicyProperties, durable);
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
+ // ToDo: totally replace tokenizerFactory with full-text config
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.isPartitioned = isPartitioned;
this.invertedIndexFields = invertedIndexFields;
this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
@@ -77,8 +84,8 @@
filterTypeTraits, filterCmpFactories, filterFields, opTrackerProvider, ioOpCallbackFactory,
pageWriteCallbackFactory, metadataPageManagerFactory, vbcProvider, ioSchedulerProvider,
mergePolicyFactory, mergePolicyProperties, durable, tokenTypeTraits, tokenCmpFactories,
- tokenizerFactory, isPartitioned, invertedIndexFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
+ tokenizerFactory, fullTextConfigEvaluatorFactory, isPartitioned, invertedIndexFields,
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, bloomFilterFalsePositiveRate);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
index 13a649d..2cee54d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorDescriptor.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
public class LSMInvertedIndexSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -44,6 +45,7 @@
private final boolean isFullTextSearchQuery;
private final IIndexDataflowHelperFactory indexHelperFactory;
private final IBinaryTokenizerFactory queryTokenizerFactory;
+ private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
private final boolean retainInput;
private final boolean retainMissing;
private final IMissingWriterFactory missingWriterFactory;
@@ -54,14 +56,16 @@
public LSMInvertedIndexSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int queryField, IIndexDataflowHelperFactory indexHelperFactory,
- IBinaryTokenizerFactory queryTokenizerFactory, IInvertedIndexSearchModifierFactory searchModifierFactory,
- boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
- ISearchOperationCallbackFactory searchCallbackFactory, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
- int frameLimit) {
+ IBinaryTokenizerFactory queryTokenizerFactory,
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ IInvertedIndexSearchModifierFactory searchModifierFactory, boolean retainInput, boolean retainMissing,
+ IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean isFullTextSearchQuery, int numOfFields,
+ boolean appendIndexFilter, int frameLimit) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.queryTokenizerFactory = queryTokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.queryField = queryField;
this.retainInput = retainInput;
this.retainMissing = retainMissing;
@@ -84,7 +88,7 @@
return new LSMInvertedIndexSearchOperatorNodePushable(ctx,
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), partition, minFilterFieldIndexes,
maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing, missingWriterFactory,
- searchCallbackFactory, searchModifier, queryTokenizerFactory, queryField, isFullTextSearchQuery,
- numOfFields, appendIndexFilter, frameLimit);
+ searchCallbackFactory, searchModifier, queryTokenizerFactory, fullTextConfigEvaluatorFactory,
+ queryField, isFullTextSearchQuery, numOfFields, appendIndexFilter, frameLimit);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
index a27dea7..e4dbb43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/dataflow/LSMInvertedIndexSearchOperatorNodePushable.java
@@ -34,7 +34,10 @@
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.ISearchPredicate;
@@ -43,6 +46,7 @@
protected final IInvertedIndexSearchModifier searchModifier;
protected final IBinaryTokenizerFactory binaryTokenizerFactory;
+ protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
protected final int queryFieldIndex;
protected final int numOfFields;
// Keeps the information whether the given query is a full-text search or not.
@@ -57,12 +61,14 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
IInvertedIndexSearchModifier searchModifier, IBinaryTokenizerFactory binaryTokenizerFactory,
- int queryFieldIndex, boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter,
- int frameLimit) throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, int queryFieldIndex,
+ boolean isFullTextSearchQuery, int numOfFields, boolean appendIndexFilter, int frameLimit)
+ throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter);
this.searchModifier = searchModifier;
this.binaryTokenizerFactory = binaryTokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.queryFieldIndex = queryFieldIndex;
this.isFullTextSearchQuery = isFullTextSearchQuery;
// If retainInput is true, the frameTuple is created in IndexSearchOperatorNodePushable.open().
@@ -79,7 +85,11 @@
@Override
protected ISearchPredicate createSearchPredicate() {
- return new InvertedIndexSearchPredicate(binaryTokenizerFactory.createTokenizer(), searchModifier, minFilterKey,
+ IBinaryTokenizer tokenizer = binaryTokenizerFactory.createTokenizer();
+ IFullTextConfigEvaluator fullTextConfigEvaluator =
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+
+ return new InvertedIndexSearchPredicate(tokenizer, fullTextConfigEvaluator, searchModifier, minFilterKey,
maxFilterKey, isFullTextSearchQuery);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java
new file mode 100644
index 0000000..50d87b5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluator.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+public abstract class AbstractFullTextFilterEvaluator implements IFullTextFilterEvaluator {
+
+ protected final String name;
+ protected final FullTextFilterType type;
+
+ public AbstractFullTextFilterEvaluator(String name, FullTextFilterType type) {
+ this.name = name;
+ this.type = type;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public FullTextFilterType getFilterType() {
+ return type;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
new file mode 100644
index 0000000..4fa27ca
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/AbstractFullTextFilterEvaluatorFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+public abstract class AbstractFullTextFilterEvaluatorFactory implements IFullTextFilterEvaluatorFactory {
+ protected final String name;
+ protected final FullTextFilterType type;
+ protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ protected AbstractFullTextFilterEvaluatorFactory(String name, FullTextFilterType type) {
+ this.name = name;
+ this.type = type;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java
new file mode 100644
index 0000000..6805c43
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluator.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+import com.google.common.collect.ImmutableList;
+
+// FullTextConfigEvaluator is a run-time evaluator while the IFullTextConfigDescriptor is a compile-time descriptor
+//
+// The descriptor is responsible for serialization (i.e. distributed from the CC to NC)
+// and Metadata translator (i.e. be written to the metadata catalog)
+// And the analyzer is to process the tokens in each NC at run-time
+public class FullTextConfigEvaluator implements IFullTextConfigEvaluator {
+
+ // name is not necessary for run-time token processing, however, let's keep this field for debugging purpose
+ // The name of the evaluator is supposed to have the dataverse name and the name of the full-text config descriptor
+ private final String name;
+ // Due to the limitation of existing code (e.g. some call sites assume the input has a Asterix type tag while some don't),
+ // we may need to replace the tokenizer on-the-fly in certain call sites.
+ // So this field is not final
+ private IBinaryTokenizer tokenizer;
+ private final ImmutableList<IFullTextFilterEvaluator> filters;
+
+ private IToken currentToken;
+ private IToken nextToken;
+
+ public FullTextConfigEvaluator(String name, TokenizerCategory tokenizerCategory,
+ ImmutableList<IFullTextFilterEvaluator> filters) {
+ this.name = name;
+ this.filters = filters;
+
+ switch (tokenizerCategory) {
+ case WORD:
+ // Currently, the tokenizer will be set later after the analyzer created
+ // This is because the tokenizer logic is complex,
+ // and we are already using a dedicated tokenizer factory to create tokenizer.
+ // One tricky part of tokenizer is that it can be call-site specific, e.g. the string in some call-site
+ // has the ATypeTag.String in the beginning of its byte array, and some doesn't,
+ // so if we only know the category of the tokenizer, e.g. a WORD tokenizer,
+ // we still cannot create a suitable tokenizer here as the tokenizer factory does.
+ //
+ // Finally we should get rid of the dedicated tokenizer factory and put its related logic
+ // in the full-text descriptor and analyzer
+ this.tokenizer = null;
+ break;
+ case NGRAM:
+ default:
+ throw new IllegalArgumentException();
+ }
+ }
+
+ @Override
+ public IBinaryTokenizer getTokenizer() {
+ return tokenizer;
+ }
+
+ @Override
+ public TokenizerCategory getTokenizerCategory() {
+ return tokenizer.getTokenizerCategory();
+ }
+
+ @Override
+ public void setTokenizer(IBinaryTokenizer tokenizer) {
+ this.tokenizer = tokenizer;
+ }
+
+ @Override
+ public String getName() {
+ return name;
+ }
+
+ @Override
+ public void reset(byte[] data, int start, int length) {
+ currentToken = null;
+ nextToken = null;
+ tokenizer.reset(data, start, length);
+ }
+
+ @Override
+ public IToken getToken() {
+ return currentToken;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (nextToken != null) {
+ return true;
+ }
+
+ while (tokenizer.hasNext()) {
+ tokenizer.next();
+ IToken candidateToken = tokenizer.getToken();
+ for (IFullTextFilterEvaluator filter : filters) {
+ // ToDo: Tokenizer of TokenizerType.List would return strings starting with the length,
+ // e.g. 8database where 8 is the length
+ // Should we let TokenizerType.List returns the same thing as TokenizerType.String to make things easier?
+ // If so, we need to remove the length in the input string in all the call site of the tokenizer
+ // Otherwise, filters need tokenizer.getTokenizerType to decide if they need to remove the length themselves
+ candidateToken = filter.processToken(tokenizer.getTokenizerType(), candidateToken);
+ // null means the token is removed, i.e. it is a stopword
+ if (candidateToken == null) {
+ break;
+ }
+ }
+
+ if (candidateToken != null) {
+ nextToken = candidateToken;
+ break;
+ }
+ }
+
+ return nextToken != null;
+ }
+
+ @Override
+ public void next() {
+ currentToken = nextToken;
+ nextToken = null;
+ }
+
+ @Override
+ public int getTokensCount() {
+ return tokenizer.getTokensCount();
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java
new file mode 100644
index 0000000..cb7c86a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextConfigEvaluatorFactory.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.ImmutableList;
+
+public class FullTextConfigEvaluatorFactory implements IFullTextConfigEvaluatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private final String name;
+ private final TokenizerCategory tokenizerCategory;
+ private final ImmutableList<IFullTextFilterEvaluatorFactory> filters;
+
+ public FullTextConfigEvaluatorFactory(String name, TokenizerCategory tokenizerCategory,
+ ImmutableList<IFullTextFilterEvaluatorFactory> filters) {
+ this.name = name;
+ this.tokenizerCategory = tokenizerCategory;
+ this.filters = filters;
+ }
+
+ @Override
+ public IFullTextConfigEvaluator createFullTextConfigEvaluator() {
+ ImmutableList.Builder<IFullTextFilterEvaluator> filterEvaluatorsBuilder = ImmutableList.builder();
+ for (IFullTextFilterEvaluatorFactory factory : filters) {
+ filterEvaluatorsBuilder.add(factory.createFullTextFilterEvaluator());
+ }
+ return new FullTextConfigEvaluator(name, tokenizerCategory, filterEvaluatorsBuilder.build());
+ }
+
+ public static IFullTextConfigEvaluatorFactory getDefaultFactory() {
+ return new FullTextConfigEvaluatorFactory("default_config_evaluator_factory", TokenizerCategory.WORD,
+ ImmutableList.of());
+ }
+
+ private static final String FIELD_NAME = "name";
+ private static final String FIELD_TOKENIZER_CATEGORY = "tokenizerCategory";
+ private static final String FIELD_FILTERS = "filters";
+ protected static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ final ObjectNode json = registry.getClassIdentifier(getClass(), serialVersionUID);
+ json.put(FIELD_NAME, name);
+ json.put(FIELD_TOKENIZER_CATEGORY, tokenizerCategory.toString());
+
+ final ArrayNode filterArray = OBJECT_MAPPER.createArrayNode();
+ for (IFullTextFilterEvaluatorFactory filter : filters) {
+ filterArray.add(filter.toJson(registry));
+ }
+ json.set(FIELD_FILTERS, filterArray);
+
+ return json;
+ }
+
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ throws HyracksDataException {
+ final String name = json.get(FIELD_NAME).asText();
+ final String tokenizerCategoryStr = json.get(FIELD_TOKENIZER_CATEGORY).asText();
+ TokenizerCategory tc = TokenizerCategory.getEnumIgnoreCase(tokenizerCategoryStr);
+
+ ArrayNode filtersJsonNode = (ArrayNode) json.get(FIELD_FILTERS);
+ ImmutableList.Builder<IFullTextFilterEvaluatorFactory> filtersBuilder = ImmutableList.builder();
+ for (int i = 0; i < filtersJsonNode.size(); i++) {
+ filtersBuilder.add((IFullTextFilterEvaluatorFactory) registry.deserialize(filtersJsonNode.get(i)));
+ }
+ return new FullTextConfigEvaluatorFactory(name, tc, filtersBuilder.build());
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java
new file mode 100644
index 0000000..abfd753
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/FullTextFilterType.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.commons.lang3.EnumUtils;
+
+public enum FullTextFilterType {
+ STOPWORDS("Stopwords"),
+ SYNONYM("Synonym"),
+ STEMMER("Stemmer");
+
+ private final String value;
+
+ FullTextFilterType(String value) {
+ this.value = value;
+ }
+
+ public String getValue() {
+ return value;
+ }
+
+ public static FullTextFilterType getEnumIgnoreCase(String str) {
+ FullTextFilterType type = EnumUtils.getEnumIgnoreCase(FullTextFilterType.class, str);
+
+ if (type == null) {
+ throw new IllegalArgumentException("Cannot convert string " + str + " to FullTextFilterType!");
+ }
+ return type;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java
new file mode 100644
index 0000000..94ee693
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluator.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+
+public interface IFullTextConfigEvaluator {
+ String getName();
+
+ TokenizerCategory getTokenizerCategory();
+
+ void setTokenizer(IBinaryTokenizer tokenizer);
+
+ void reset(byte[] data, int start, int length);
+
+ // ToDo: wrap the tokenizer and filters into a dedicated Java class
+ // so that at runtime the operators (evaluators) don't touch the usedByIndices filed
+ // That means, the usedByIndices field should be modified via MetadataManager only at compile time
+ IBinaryTokenizer getTokenizer();
+
+ IToken getToken();
+
+ boolean hasNext();
+
+ void next();
+
+ // Get the total number of tokens
+ // Currently, it returns the number of tokens in the original text, that means stopwords are still counted
+ int getTokensCount();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java
new file mode 100644
index 0000000..9469b66
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextConfigEvaluatorFactory.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+public interface IFullTextConfigEvaluatorFactory extends Serializable, IJsonSerializable {
+ IFullTextConfigEvaluator createFullTextConfigEvaluator();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java
new file mode 100644
index 0000000..9c36ea3
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluator.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo;
+
+// Full-text filter evaluator that process tokens
+// Such an evaluator is created via IFullTextFilterEvaluatorFactory,
+// and the run-time evaluator factory is created from IFullTextFilterDescriptor which is a compile-time concept.
+public interface IFullTextFilterEvaluator {
+ IToken processToken(TokenizerInfo.TokenizerType tokenizerType, IToken token);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java
new file mode 100644
index 0000000..7f77e0f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/IFullTextFilterEvaluatorFactory.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.io.IJsonSerializable;
+
+// This full-text filter evaluator factory would to be stored in the index local resource,
+// so it needs to be IJsonSerializable.
+// Also, it would to be distributed from CC (compile-time) to NC (run-time), so it needs to be Serializable.
+//
+// Such a IFullTextFilterEvaluatorFactory should always be wrapped in a IFullTextConfigEvaluatorFactory
+// because filter cannot live without a config: a full-text config is responsible to tokenize strings
+// and then feed the tokens into the filters.
+public interface IFullTextFilterEvaluatorFactory extends IJsonSerializable, Serializable {
+ IFullTextFilterEvaluator createFullTextFilterEvaluator();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java
new file mode 100644
index 0000000..da5fbb0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluator.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import java.util.List;
+
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterEvaluator extends AbstractFullTextFilterEvaluator {
+
+ private final ImmutableList<String> stopwordList;
+
+ public StopwordsFullTextFilterEvaluator(String name, ImmutableList<String> stopwordList) {
+ super(name, FullTextFilterType.STOPWORDS);
+ this.stopwordList = stopwordList;
+ }
+
+ public List<String> getStopwordList() {
+ return stopwordList;
+ }
+
+ @Override
+ public IToken processToken(TokenizerInfo.TokenizerType tokenizerType, IToken token) {
+ int start = token.getStartOffset();
+ int length = token.getTokenLength();
+
+ // The List tokenizer returns token starting with the token length,
+ // e.g. 8database where the byte of value 8 means the token has a length of 8
+ // We need to skip the length to fetch the pure string (e.g. "database" without 8)
+ if (tokenizerType == TokenizerInfo.TokenizerType.LIST) {
+ int numBytesToStoreLength = UTF8StringUtil
+ .getNumBytesToStoreLength(UTF8StringUtil.getUTFLength(token.getData(), token.getStartOffset()));
+ start += numBytesToStoreLength;
+ length -= numBytesToStoreLength;
+ }
+
+ String str = UTF8StringUtil.getUTF8StringInArray(token.getData(), start, length);
+ if (stopwordList.contains(str)) {
+ return null;
+ }
+
+ return token;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java
new file mode 100644
index 0000000..b31aa17
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/StopwordsFullTextFilterEvaluatorFactory.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IJsonSerializable;
+import org.apache.hyracks.api.io.IPersistedResourceRegistry;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.google.common.collect.ImmutableList;
+
+public class StopwordsFullTextFilterEvaluatorFactory extends AbstractFullTextFilterEvaluatorFactory {
+ private static final long serialVersionUID = 1L;
+
+ private final ImmutableList<String> stopwordList;
+ private static final String STOPWORDS_FILTER_NAME = "stopwordsFilterName";
+ private static final String STOPWORDS_LIST = "stopwordsList";
+
+ public StopwordsFullTextFilterEvaluatorFactory(String name, ImmutableList<String> stopwordList) {
+ super(name, FullTextFilterType.STOPWORDS);
+ this.stopwordList = stopwordList;
+ }
+
+ @Override
+ public IFullTextFilterEvaluator createFullTextFilterEvaluator() {
+ return new StopwordsFullTextFilterEvaluator(name, stopwordList);
+ }
+
+ // ToDo: extract the common logics to a dedicated helper or utilization class after more filters are implemented
+ @Override
+ public JsonNode toJson(IPersistedResourceRegistry registry) throws HyracksDataException {
+ final ObjectNode json = registry.getClassIdentifier(getClass(), this.serialVersionUID);
+ json.put(STOPWORDS_FILTER_NAME, name);
+
+ ArrayNode stopwordsArrayNode = OBJECT_MAPPER.createArrayNode();
+ for (String s : stopwordList) {
+ stopwordsArrayNode.add(s);
+ }
+ json.set(STOPWORDS_LIST, stopwordsArrayNode);
+
+ return json;
+ }
+
+ public static IJsonSerializable fromJson(IPersistedResourceRegistry registry, JsonNode json)
+ throws HyracksDataException {
+ final String name = json.get(STOPWORDS_FILTER_NAME).asText();
+
+ // ToDo: create a new function to extract a list from json
+ ImmutableList.Builder<String> stopwordsBuilder = ImmutableList.<String> builder();
+ JsonNode stopwordsArrayNode = json.get(STOPWORDS_LIST);
+ for (int i = 0; i < stopwordsArrayNode.size(); i++) {
+ stopwordsBuilder.add(stopwordsArrayNode.get(i).asText());
+ }
+ ImmutableList<String> stopwords = stopwordsBuilder.build();
+
+ return new StopwordsFullTextFilterEvaluatorFactory(name, stopwords);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java
new file mode 100644
index 0000000..9c9b1a1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/fulltext/TokenizerCategory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.hyracks.storage.am.lsm.invertedindex.fulltext;
+
+import org.apache.commons.lang3.EnumUtils;
+
+public enum TokenizerCategory {
+ NGRAM,
+ WORD;
+
+ public static TokenizerCategory getEnumIgnoreCase(String str) {
+ return EnumUtils.getEnumIgnoreCase(TokenizerCategory.class, str);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index f8ad664..de65e3e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -60,6 +60,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
@@ -82,6 +83,7 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final IBinaryTokenizerFactory tokenizerFactory;
+ protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
// Type traits and comparators for tokens and inverted-list elements.
protected final ITypeTraits[] invListTypeTraits;
@@ -97,16 +99,17 @@
double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
- int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- ITracer tracer) throws HyracksDataException {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
+ int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, componentFactory,
componentFactory, filterFrameFactory, filterManager, filterFields, durable, filterHelper,
invertedIndexFields, tracer);
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
this.tokenTypeTraits = tokenTypeTraits;
@@ -406,6 +409,7 @@
VirtualFreePageManager virtualFreePageManager, int id) throws HyracksDataException {
return InvertedIndexUtils.createInMemoryBTreeInvertedindex(virtualBufferCache, virtualFreePageManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ fullTextConfigEvaluatorFactory,
ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
index 23d6221..de1fc49 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/PartitionedLSMInvertedIndex.java
@@ -37,6 +37,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFilterManager;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -51,16 +52,16 @@
double bloomFilterFalsePositiveRate, IBufferCache diskBufferCache, ILSMIndexFileManager fileManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
- ILSMPageWriteCallbackFactory pageWriteCallbackFactory, int[] invertedIndexFields, int[] filterFields,
- int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- ITracer tracer) throws HyracksDataException {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
+ int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
+ int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory, filterManager,
bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker, ioScheduler,
- ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, mergePolicy,
+ opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
+ filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
@Override
@@ -68,7 +69,8 @@
VirtualFreePageManager virtualFreePageManager, int id) throws HyracksDataException {
return InvertedIndexUtils.createPartitionedInMemoryBTreeInvertedindex(virtualBufferCache,
virtualFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories,
- tokenizerFactory, ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
+ tokenizerFactory, fullTextConfigEvaluatorFactory,
+ ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_vocab_" + id));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index 45e08ce..6998861 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -50,6 +51,7 @@
protected final ITypeTraits[] invListTypeTraits;
protected final IBinaryComparatorFactory[] invListCmpFactories;
protected final IBinaryTokenizerFactory tokenizerFactory;
+ protected final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
protected final ITypeTraits[] btreeTypeTraits;
protected final IBinaryComparatorFactory[] btreeCmpFactories;
@@ -57,12 +59,14 @@
public InMemoryInvertedIndex(IBufferCache virtualBufferCache, IPageManager virtualFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws HyracksDataException {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ FileReference btreeFileRef) throws HyracksDataException {
this.tokenTypeTraits = tokenTypeTraits;
this.tokenCmpFactories = tokenCmpFactories;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
// BTree tuples: <tokens, inverted-list elements>.
int numBTreeFields = tokenTypeTraits.length + invListTypeTraits.length;
btreeTypeTraits = new ITypeTraits[numBTreeFields];
@@ -174,7 +178,8 @@
@Override
public InMemoryInvertedIndexAccessor createAccessor(IIndexAccessParameters iap) throws HyracksDataException {
return new InMemoryInvertedIndexAccessor(this,
- new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory),
+ new InMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory,
+ fullTextConfigEvaluatorFactory),
(IHyracksTaskContext) iap.getParameters().get(HyracksConstants.HYRACKS_TASK_CONTEXT));
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
index 0457b46..1a52d2e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexOpContext.java
@@ -27,6 +27,8 @@
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexTokenizingTupleIterator;
@@ -46,14 +48,16 @@
// To generate in-memory BTree tuples for insertions.
private final IBinaryTokenizerFactory tokenizerFactory;
+ private final IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory;
private InvertedIndexTokenizingTupleIterator tupleIter;
private boolean destroyed = false;
InMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory) {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) {
this.btree = btree;
this.tokenCmpFactories = tokenCmpFactories;
this.tokenizerFactory = tokenizerFactory;
+ this.fullTextConfigEvaluatorFactory = fullTextConfigEvaluatorFactory;
}
@Override
@@ -93,9 +97,11 @@
}
protected void setTokenizingTupleIterator() {
- IBinaryTokenizer tokenizer = getTokenizerFactory().createTokenizer();
+ IBinaryTokenizer tokenizer = tokenizerFactory.createTokenizer();
+ IFullTextConfigEvaluator fullTextConfigEvaluator =
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
tupleIter = new InvertedIndexTokenizingTupleIterator(tokenCmpFactories.length,
- btree.getFieldCount() - tokenCmpFactories.length, tokenizer);
+ btree.getFieldCount() - tokenCmpFactories.length, tokenizer, fullTextConfigEvaluator);
}
public InvertedIndexTokenizingTupleIterator getTupleIter() {
@@ -122,6 +128,10 @@
return tokenizerFactory;
}
+ public IFullTextConfigEvaluatorFactory getFullTextConfigEvaluatorFactory() {
+ return fullTextConfigEvaluatorFactory;
+ }
+
public void setTupleIter(InvertedIndexTokenizingTupleIterator tupleIter) {
this.tupleIter = tupleIter;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
index 806bbf5..bd9ce60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndex.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearcher;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IPartitionedInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedListPartitions;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.PartitionedTOccurrenceSearcher;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
@@ -47,9 +48,10 @@
public PartitionedInMemoryInvertedIndex(IBufferCache memBufferCache, IPageManager memFreePageManager,
ITypeTraits[] invListTypeTraits, IBinaryComparatorFactory[] invListCmpFactories,
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory, FileReference btreeFileRef) throws HyracksDataException {
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory,
+ FileReference btreeFileRef) throws HyracksDataException {
super(memBufferCache, memFreePageManager, invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, btreeFileRef);
+ tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
}
@Override
@@ -86,8 +88,8 @@
@Override
public PartitionedInMemoryInvertedIndexAccessor createAccessor(IIndexAccessParameters iap)
throws HyracksDataException {
- return new PartitionedInMemoryInvertedIndexAccessor(this,
- new PartitionedInMemoryInvertedIndexOpContext(btree, tokenCmpFactories, tokenizerFactory), iap);
+ return new PartitionedInMemoryInvertedIndexAccessor(this, new PartitionedInMemoryInvertedIndexOpContext(btree,
+ tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory), iap);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
index 9cfaf7a..6b6401a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/PartitionedInMemoryInvertedIndexOpContext.java
@@ -21,6 +21,8 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.storage.am.btree.impls.BTree;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.PartitionedInvertedIndexTokenizingTupleIterator;
@@ -28,13 +30,15 @@
public class PartitionedInMemoryInvertedIndexOpContext extends InMemoryInvertedIndexOpContext {
public PartitionedInMemoryInvertedIndexOpContext(BTree btree, IBinaryComparatorFactory[] tokenCmpFactories,
- IBinaryTokenizerFactory tokenizerFactory) {
- super(btree, tokenCmpFactories, tokenizerFactory);
+ IBinaryTokenizerFactory tokenizerFactory, IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory) {
+ super(btree, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory);
}
protected void setTokenizingTupleIterator() {
IBinaryTokenizer tokenizer = getTokenizerFactory().createTokenizer();
+ IFullTextConfigEvaluator fullTextConfigEvaluator =
+ getFullTextConfigEvaluatorFactory().createFullTextConfigEvaluator();
setTupleIter(new PartitionedInvertedIndexTokenizingTupleIterator(tokenCmpFactories.length,
- btree.getFieldCount() - tokenCmpFactories.length, tokenizer));
+ btree.getFieldCount() - tokenCmpFactories.length, tokenizer, fullTextConfigEvaluator));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
index 1a08fd0..2fb620c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/AbstractTOccurrenceSearcher.java
@@ -45,8 +45,8 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IObjectFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
@@ -126,25 +126,27 @@
protected void tokenizeQuery(InvertedIndexSearchPredicate searchPred) throws HyracksDataException {
ITupleReference queryTuple = searchPred.getQueryTuple();
int queryFieldIndex = searchPred.getQueryFieldIndex();
- IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer();
+ IFullTextConfigEvaluator fullTextAnalyzer = searchPred.getFullTextConfigEvaluator();
+ fullTextAnalyzer.setTokenizer(searchPred.getQueryTokenizer());
+
// Is this a full-text query?
// Then, the last argument is conjunctive or disjunctive search option, not a query text.
// Thus, we need to remove the last argument.
boolean isFullTextSearchQuery = searchPred.getIsFullTextSearchQuery();
// Get the type of query tokenizer.
- TokenizerType queryTokenizerType = queryTokenizer.getTokenizerType();
+ TokenizerType queryTokenizerType = fullTextAnalyzer.getTokenizer().getTokenizerType();
int tokenCountInOneField = 0;
queryTokenAppender.reset(queryTokenFrame, true);
- queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
+ fullTextAnalyzer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
queryTuple.getFieldLength(queryFieldIndex));
- while (queryTokenizer.hasNext()) {
- queryTokenizer.next();
+ while (fullTextAnalyzer.hasNext()) {
+ fullTextAnalyzer.next();
queryTokenBuilder.reset();
tokenCountInOneField++;
try {
- IToken token = queryTokenizer.getToken();
+ IToken token = fullTextAnalyzer.getToken();
// For the full-text search, we don't support a phrase search yet.
// So, each field should have only one token.
// If it's a list, it can have multiple keywords in it. But, each keyword should not be a phrase.
@@ -161,6 +163,7 @@
}
}
+ // Includes the length of the string, e.g. 8database where 8 (of type byte instead of char) is the length of "database"
token.serializeToken(queryTokenBuilder.getFieldData());
queryTokenBuilder.addFieldEndOffset();
// WARNING: assuming one frame is big enough to hold all tokens
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
index 4ee685b..3cf9066 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/search/InvertedIndexSearchPredicate.java
@@ -22,6 +22,7 @@
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.impls.AbstractSearchPredicate;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -31,21 +32,27 @@
private ITupleReference queryTuple;
private int queryFieldIndex;
private final IBinaryTokenizer queryTokenizer;
+ private final IFullTextConfigEvaluator fullTextConfigEvaluator;
private final IInvertedIndexSearchModifier searchModifier;
// Keeps the information whether the given query is a full-text search or not.
// We need to have this information to stop the search process since we don't allow a phrase search yet.
private boolean isFullTextSearchQuery;
- public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier) {
+ // Used for test only
+ public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer,
+ IFullTextConfigEvaluator fullTextConfigEvaluator, IInvertedIndexSearchModifier searchModifier) {
this.queryTokenizer = queryTokenizer;
+ this.fullTextConfigEvaluator = fullTextConfigEvaluator;
this.searchModifier = searchModifier;
this.isFullTextSearchQuery = false;
}
- public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier,
+ public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer,
+ IFullTextConfigEvaluator fullTextConfigEvaluator, IInvertedIndexSearchModifier searchModifier,
ITupleReference minFilterTuple, ITupleReference maxFilterTuple, boolean isFullTextSearchQuery) {
super(minFilterTuple, maxFilterTuple);
this.queryTokenizer = queryTokenizer;
+ this.fullTextConfigEvaluator = fullTextConfigEvaluator;
this.searchModifier = searchModifier;
this.isFullTextSearchQuery = isFullTextSearchQuery;
}
@@ -82,6 +89,10 @@
return queryTokenizer;
}
+ public IFullTextConfigEvaluator getFullTextConfigEvaluator() {
+ return fullTextConfigEvaluator;
+ }
+
@Override
public MultiComparator getLowKeyComparator() {
// TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
index 3a5224c..df06525 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/DelimitedUTF8StringBinaryTokenizer.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
import org.apache.hyracks.util.string.UTF8StringUtil;
@@ -51,6 +52,8 @@
return byteIndex < sentenceEndOffset;
}
+ // ToDo: current solution (where a white list defined for non-separator chars) is not suitable for double-surrogate utf-8 chars
+ // which are always judged as separators
public static boolean isSeparator(char c) {
return !(Character.isLetterOrDigit(c) || Character.getType(c) == Character.OTHER_LETTER
|| Character.getType(c) == Character.OTHER_NUMBER);
@@ -119,4 +122,9 @@
public TokenizerType getTokenizerType() {
return TokenizerType.STRING;
}
+
+ @Override
+ public TokenizerCategory getTokenizerCategory() {
+ return TokenizerCategory.WORD;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
index 6a7da02..0e7f30d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/IBinaryTokenizer.java
@@ -19,20 +19,24 @@
package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
public interface IBinaryTokenizer {
- public IToken getToken();
+ IToken getToken();
- public boolean hasNext();
+ boolean hasNext();
- public void next();
+ void next();
- public void reset(byte[] data, int start, int length);
+ void reset(byte[] data, int start, int length);
// Get the total number of tokens
- public short getTokensCount();
+ short getTokensCount();
- // Get the tokenizer types
- public TokenizerType getTokenizerType();
+ // Get the tokenizer types: String or List
+ TokenizerType getTokenizerType();
+
+ // WORD or NGRAM tokenizer
+ TokenizerCategory getTokenizerCategory();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
index 711a82f..0ab1662 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tokenizers/NGramUTF8StringBinaryTokenizer.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.TokenizerInfo.TokenizerType;
import org.apache.hyracks.util.string.UTF8StringUtil;
@@ -136,4 +137,9 @@
public TokenizerType getTokenizerType() {
return TokenizerType.STRING;
}
+
+ @Override
+ public TokenizerCategory getTokenizerCategory() {
+ return TokenizerCategory.NGRAM;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
index 4cf0c55..19d6f11 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexTokenizingTupleIterator.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
@@ -36,38 +37,43 @@
protected final int invListFieldCount;
protected final ArrayTupleBuilder tupleBuilder;
protected final ArrayTupleReference tupleReference;
- protected final IBinaryTokenizer tokenizer;
+ protected final IFullTextConfigEvaluator fullTextConfigEvaluator;
protected ITupleReference inputTuple;
- public InvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
- IBinaryTokenizer tokenizer) {
+ public InvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount, IBinaryTokenizer tokenizer,
+ IFullTextConfigEvaluator fullTextConfigEvaluator) {
this.invListFieldCount = invListFieldCount;
this.tupleBuilder = new ArrayTupleBuilder(tokensFieldCount + invListFieldCount);
this.tupleReference = new ArrayTupleReference();
- this.tokenizer = tokenizer;
+ this.fullTextConfigEvaluator = fullTextConfigEvaluator;
+
+ // ToDo: check the codes in upper layer to see if we can remove tokenizer to use fullTextConfig instead
+ this.fullTextConfigEvaluator.setTokenizer(tokenizer);
}
public void reset(ITupleReference inputTuple) {
this.inputTuple = inputTuple;
- tokenizer.reset(inputTuple.getFieldData(DOC_FIELD_INDEX), inputTuple.getFieldStart(DOC_FIELD_INDEX),
- inputTuple.getFieldLength(DOC_FIELD_INDEX));
+ fullTextConfigEvaluator.reset(inputTuple.getFieldData(DOC_FIELD_INDEX),
+ inputTuple.getFieldStart(DOC_FIELD_INDEX), inputTuple.getFieldLength(DOC_FIELD_INDEX));
}
public boolean hasNext() {
- return tokenizer.hasNext();
+ return fullTextConfigEvaluator.hasNext();
}
public void next() throws HyracksDataException {
- tokenizer.next();
- IToken token = tokenizer.getToken();
+ fullTextConfigEvaluator.next();
+ IToken token = fullTextConfigEvaluator.getToken();
+
tupleBuilder.reset();
- // Add token field.
try {
+ // Add token field.
token.serializeToken(tupleBuilder.getFieldData());
} catch (IOException e) {
throw HyracksDataException.create(e);
}
tupleBuilder.addFieldEndOffset();
+
// Add inverted-list element fields.
for (int i = 0; i < invListFieldCount; i++) {
tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
index a9a01e9..dcde832 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/InvertedIndexUtils.java
@@ -52,6 +52,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilderFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListTupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexFileManager;
@@ -83,18 +84,21 @@
IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- FileReference btreeFileRef) throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
+ throws HyracksDataException {
return new InMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
+ tokenTypeTraits, tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory, btreeFileRef);
}
public static InMemoryInvertedIndex createPartitionedInMemoryBTreeInvertedindex(IBufferCache memBufferCache,
IPageManager virtualFreePageManager, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- FileReference btreeFileRef) throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, FileReference btreeFileRef)
+ throws HyracksDataException {
return new PartitionedInMemoryInvertedIndex(memBufferCache, virtualFreePageManager, invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, btreeFileRef);
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, btreeFileRef);
}
public static OnDiskInvertedIndex createOnDiskInvertedIndex(IIOManager ioManager, IBufferCache bufferCache,
@@ -138,8 +142,9 @@
List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, IBufferCache diskBufferCache,
+ String absoluteOnDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -179,17 +184,19 @@
return new LSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper, filterFrameFactory,
filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
- invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
- ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+ pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
List<IVirtualBufferCache> virtualBufferCaches, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory,
- IBufferCache diskBufferCache, String absoluteOnDiskDir, double bloomFilterFalsePositiveRate,
- ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, IBufferCache diskBufferCache,
+ String absoluteOnDiskDir, double bloomFilterFalsePositiveRate, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, ILSMIOOperationScheduler ioScheduler,
ILSMIOOperationCallbackFactory ioOpCallbackFactory, ILSMPageWriteCallbackFactory pageWriteCallbackFactory,
int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps,
@@ -229,8 +236,9 @@
return new PartitionedLSMInvertedIndex(ioManager, virtualBufferCaches, componentFactory, filterHelper,
filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, pageWriteCallbackFactory, invertedIndexFields,
- filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
+ fullTextConfigEvaluatorFactory, mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory,
+ pageWriteCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
public static boolean checkTypeTraitsAllFixed(ITypeTraits[] typeTraits) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
index c44dffb..783afc6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/PartitionedInvertedIndexTokenizingTupleIterator.java
@@ -23,6 +23,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
@@ -32,28 +33,33 @@
protected short numTokens = 0;
public PartitionedInvertedIndexTokenizingTupleIterator(int tokensFieldCount, int invListFieldCount,
- IBinaryTokenizer tokenizer) {
- super(tokensFieldCount, invListFieldCount, tokenizer);
+ IBinaryTokenizer tokenizer, IFullTextConfigEvaluator fullTextConfigEvaluator) {
+ super(tokensFieldCount, invListFieldCount, tokenizer, fullTextConfigEvaluator);
}
+ @Override
public void reset(ITupleReference inputTuple) {
super.reset(inputTuple);
// Run through the tokenizer once to get the total number of tokens.
numTokens = 0;
- while (tokenizer.hasNext()) {
- tokenizer.next();
+ while (fullTextConfigEvaluator.hasNext()) {
+ fullTextConfigEvaluator.next();
numTokens++;
}
super.reset(inputTuple);
}
+ @Override
public void next() throws HyracksDataException {
- tokenizer.next();
- IToken token = tokenizer.getToken();
+ fullTextConfigEvaluator.next();
+ IToken token = fullTextConfigEvaluator.getToken();
+
tupleBuilder.reset();
try {
// Add token field.
token.serializeToken(tupleBuilder.getFieldData());
+
+ // Different from super.next(): here we write the numTokens
tupleBuilder.addFieldEndOffset();
// Add field with number of tokens.
tupleBuilder.getDataOutput().writeShort(numTokens);
@@ -61,6 +67,7 @@
} catch (IOException e) {
throw HyracksDataException.create(e);
}
+
// Add inverted-list element fields.
for (int i = 0; i < invListFieldCount; i++) {
tupleBuilder.addField(inputTuple.getFieldData(i + 1), inputTuple.getFieldStart(i + 1),
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
index 7a70f6f..01b478b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
index a1cbdb1..46b21e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
index d0dfbf4..2e0ac6c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
<license>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
index db7000c..341c852 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
index 75ffbe3..efbfd87 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-dataflow-common-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
index 9f10473..ce2dde7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-bloomfilter-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
index fb82d56..68ccad6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<properties>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
index da4a9c4..0aa4ed3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index f34eb72..6a87ad7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
index 4934660..79cfa61 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<artifactId>hyracks-tests</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
@@ -116,6 +116,10 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 355b45c..6f742ab 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
@@ -90,6 +91,7 @@
IBinaryTokenizerFactory tokenizerFactory = invIndex.getTokenizerFactory();
int searchModifierIndex = Math.abs(rnd.nextInt()) % TEST_SEARCH_MODIFIERS.length;
InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizerFactory.createTokenizer(),
+ FullTextConfigEvaluatorFactory.getDefaultFactory().createFullTextConfigEvaluator(),
TEST_SEARCH_MODIFIERS[searchModifierIndex]);
switch (op) {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
index 524904f..3e8db34 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestContext.java
@@ -51,6 +51,7 @@
import org.apache.hyracks.storage.am.lsm.common.freepage.VirtualFreePageManager;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluatorFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils.HyracksTaskTestContext;
import org.apache.hyracks.storage.common.IIndex;
@@ -127,9 +128,10 @@
public static LSMInvertedIndexTestContext create(LSMInvertedIndexTestHarness harness,
ISerializerDeserializer[] fieldSerdes, int tokenFieldCount, IBinaryTokenizerFactory tokenizerFactory,
- InvertedIndexType invIndexType, int[] invertedIndexFields, ITypeTraits[] filterTypeTraits,
- IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps) throws HyracksDataException {
+ IFullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory, InvertedIndexType invIndexType,
+ int[] invertedIndexFields, ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories,
+ int[] filterFields, int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps)
+ throws HyracksDataException {
ITypeTraits[] allTypeTraits = SerdeUtils.serdesToTypeTraits(fieldSerdes);
IOManager ioManager = harness.getIOManager();
IBinaryComparatorFactory[] allCmpFactories =
@@ -157,7 +159,7 @@
invIndex = InvertedIndexUtils.createInMemoryBTreeInvertedindex(harness.getVirtualBufferCaches().get(0),
new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+ fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
break;
}
case PARTITIONED_INMEMORY: {
@@ -165,7 +167,7 @@
harness.getVirtualBufferCaches().get(0),
new VirtualFreePageManager(harness.getVirtualBufferCaches().get(0)), invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
+ fullTextConfigEvaluatorFactory, ioManager.resolveAbsolutePath(harness.getOnDiskDir()));
break;
}
case ONDISK: {
@@ -183,8 +185,9 @@
case LSM: {
invIndex = InvertedIndexUtils.createLSMInvertedIndex(ioManager, harness.getVirtualBufferCaches(),
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
- harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
- harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
+ fullTextConfigEvaluatorFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
+ harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
@@ -196,9 +199,9 @@
case PARTITIONED_LSM: {
invIndex = InvertedIndexUtils.createPartitionedLSMInvertedIndex(ioManager,
harness.getVirtualBufferCaches(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
- tokenCmpFactories, tokenizerFactory, harness.getDiskBufferCache(), harness.getOnDiskDir(),
- harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(),
- harness.getOperationTracker(), harness.getIOScheduler(),
+ tokenCmpFactories, tokenizerFactory, fullTextConfigEvaluatorFactory,
+ harness.getDiskBufferCache(), harness.getOnDiskDir(), harness.getBoomFilterFalsePositiveRate(),
+ harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), harness.getPageWriteCallbackFactory(),
invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, true,
@@ -217,7 +220,8 @@
case ONDISK:
case LSM: {
indexTupleIter = new InvertedIndexTokenizingTupleIterator(invIndex.getTokenTypeTraits().length,
- invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer());
+ invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer(),
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator());
break;
}
case PARTITIONED_INMEMORY:
@@ -225,7 +229,8 @@
case PARTITIONED_LSM: {
indexTupleIter =
new PartitionedInvertedIndexTokenizingTupleIterator(invIndex.getTokenTypeTraits().length,
- invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer());
+ invIndex.getInvListTypeTraits().length, tokenizerFactory.createTokenizer(),
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator());
break;
}
default: {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index 1030088..aa9cd5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -88,6 +88,9 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedListCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.FullTextConfigEvaluatorFactory;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.IFullTextConfigEvaluator;
+import org.apache.hyracks.storage.am.lsm.invertedindex.fulltext.TokenizerCategory;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexAccessor;
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexMergeCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
@@ -113,11 +116,16 @@
import org.apache.hyracks.util.IThreadStatsCollector;
import org.apache.hyracks.util.ThreadStats;
+import com.google.common.collect.ImmutableList;
+
@SuppressWarnings("rawtypes")
public class LSMInvertedIndexTestUtils {
public static final int TEST_GRAM_LENGTH = 3;
+ public static FullTextConfigEvaluatorFactory fullTextConfigEvaluatorFactory =
+ new FullTextConfigEvaluatorFactory("test_config", TokenizerCategory.WORD, ImmutableList.of());
+
public static TupleGenerator createStringDocumentTupleGen(Random rnd) throws IOException {
IFieldValueGenerator[] fieldGens = new IFieldValueGenerator[2];
fieldGens[0] = new DocumentStringFieldValueGenerator(2, 10, 10000, rnd);
@@ -196,8 +204,9 @@
ITokenFactory tokenFactory = new UTF8WordTokenFactory();
IBinaryTokenizerFactory tokenizerFactory =
new DelimitedUTF8StringBinaryTokenizerFactory(true, false, tokenFactory);
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
- fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+ LSMInvertedIndexTestContext testCtx =
+ LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
return testCtx;
}
@@ -207,8 +216,9 @@
ITokenFactory tokenFactory = new HashedUTF8WordTokenFactory();
IBinaryTokenizerFactory tokenizerFactory =
new DelimitedUTF8StringBinaryTokenizerFactory(true, false, tokenFactory);
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
- fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+ LSMInvertedIndexTestContext testCtx =
+ LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
return testCtx;
}
@@ -218,8 +228,9 @@
ITokenFactory tokenFactory = new UTF8NGramTokenFactory();
IBinaryTokenizerFactory tokenizerFactory =
new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true, true, false, tokenFactory);
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
- fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+ LSMInvertedIndexTestContext testCtx =
+ LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
return testCtx;
}
@@ -229,8 +240,9 @@
ITokenFactory tokenFactory = new HashedUTF8NGramTokenFactory();
IBinaryTokenizerFactory tokenizerFactory =
new NGramUTF8StringBinaryTokenizerFactory(TEST_GRAM_LENGTH, true, true, false, tokenFactory);
- LSMInvertedIndexTestContext testCtx = LSMInvertedIndexTestContext.create(harness, fieldSerdes,
- fieldSerdes.length - 1, tokenizerFactory, invIndexType, null, null, null, null, null, null);
+ LSMInvertedIndexTestContext testCtx =
+ LSMInvertedIndexTestContext.create(harness, fieldSerdes, fieldSerdes.length - 1, tokenizerFactory,
+ fullTextConfigEvaluatorFactory, invIndexType, null, null, null, null, null, null);
return testCtx;
}
@@ -561,7 +573,10 @@
iap.getParameters().put(HyracksConstants.HYRACKS_TASK_CONTEXT, ctx);
IInvertedIndexAccessor accessor = (IInvertedIndexAccessor) invIndex.createAccessor(iap);
IBinaryTokenizer tokenizer = testCtx.getTokenizerFactory().createTokenizer();
- InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
+ IFullTextConfigEvaluator fullTextConfigEvaluator =
+ fullTextConfigEvaluatorFactory.createFullTextConfigEvaluator();
+ InvertedIndexSearchPredicate searchPred =
+ new InvertedIndexSearchPredicate(tokenizer, fullTextConfigEvaluator, searchModifier);
List<ITupleReference> documentCorpus = testCtx.getDocumentCorpus();
// Project away the primary-key field.
int[] fieldPermutation = new int[] { 0 };
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
index e023cc3..0df9b8a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/pom.xml
@@ -24,7 +24,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
index 020d1f6..942d424 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/pom.xml
@@ -25,7 +25,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
index 7ea46e3..29c0648 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-common-test/pom.xml
@@ -23,7 +23,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks-tests</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
<plugins>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
index 8e37cfc..19f87b2 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<licenses>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index a03e884..f908ee6 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -22,7 +22,7 @@
<parent>
<artifactId>hyracks</artifactId>
<groupId>org.apache.hyracks</groupId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
index 38de4ac..3eb8687 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/string/UTF8StringUtil.java
@@ -619,6 +619,10 @@
int count = 0;
for (int i = 0; i < strlen; i++) {
+ // ToDo: we shouldn't use str.charAt(i) to convert raw byte array to UTF-8 chars
+ // one UTF-8 char has at most four bytes, and one Java char we get via str.charAt(i) has 2 bytes
+ // In this case, a UTF-8 char may be consistent of 2 Java chars, and 1 Java char can be converted into 3 UTF-8 bytes
+ // For the emoji, it can be 6 bytes after encoded to UTF-8
c = str.charAt(i);
utflen += UTF8StringUtil.getModifiedUTF8Len(c);
}
diff --git a/hyracks-fullstack/hyracks/pom.xml b/hyracks-fullstack/hyracks/pom.xml
index c8344ba..d3fd833 100644
--- a/hyracks-fullstack/hyracks/pom.xml
+++ b/hyracks-fullstack/hyracks/pom.xml
@@ -26,7 +26,7 @@
<parent>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
</parent>
<build>
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 600d57c..0243338 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -21,7 +21,7 @@
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.hyracks</groupId>
<artifactId>apache-hyracks</artifactId>
- <version>0.3.6-SNAPSHOT</version>
+ <version>0.3.7-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hyracks-ecosystem-full-stack</name>
<url>${implementation.url}</url>
diff --git a/pom.xml b/pom.xml
index 92b77a5..1e7a240 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<artifactId>apache-asterixdb-fullstack</artifactId>
- <version>0.9.6-SNAPSHOT</version>
+ <version>0.9.7-SNAPSHOT</version>
<packaging>pom</packaging>
<name>hyracks-asterix</name>