Merge commit 'f02b43bcff1c859b933f55df9654899d025b1ed3' from release-0.9.4-pre-rc
Change-Id: I2b825e2d8b5defb67fba39f4912cfaa44111233f
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index bb76204..2aebc59 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -178,7 +178,7 @@
private void stopIfRunning(ActiveRuntimeId runtimeId, IActiveRuntime runtime)
throws HyracksDataException, InterruptedException {
- if (runtimes.remove(runtimeId) != null) {
+ if (runtimes.containsKey(runtimeId)) {
runtime.stop();
} else {
LOGGER.info("Not stopping already stopped runtime " + runtimeId);
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
deleted file mode 100644
index bc6f1b1..0000000
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/DeployedJobService.java
+++ /dev/null
@@ -1,111 +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.active;
-
-import java.time.Instant;
-import java.util.Date;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.asterix.common.transactions.ITxnIdFactory;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.DeployedJobSpecId;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-/**
- * Provides functionality for running DeployedJobSpecs
- */
-public class DeployedJobService {
-
- private static final Logger LOGGER = LogManager.getLogger();
-
- //To enable new Asterix TxnId for separate deployed job spec invocations
- private static final byte[] TRANSACTION_ID_PARAMETER_NAME = "TxnIdParameter".getBytes();
-
- //pool size one (only running one thread at a time)
- private static final int POOL_SIZE = 1;
-
- //Starts running a deployed job specification periodically with an interval of "duration" seconds
- public static ScheduledExecutorService startRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId,
- IHyracksClientConnection hcc, long duration, Map<byte[], byte[]> jobParameters, EntityId entityId,
- ITxnIdFactory txnIdFactory) {
- ScheduledExecutorService scheduledExecutorService = Executors.newScheduledThreadPool(POOL_SIZE);
- scheduledExecutorService.scheduleAtFixedRate(new Runnable() {
- @Override
- public void run() {
- try {
- if (!runRepetitiveDeployedJobSpec(distributedId, hcc, jobParameters, duration, entityId,
- txnIdFactory)) {
- scheduledExecutorService.shutdown();
- }
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "Job Failed to run for " + entityId.getExtensionName() + " "
- + entityId.getDataverse() + "." + entityId.getEntityName() + ".", e);
- }
- }
- }, duration, duration, TimeUnit.MILLISECONDS);
- return scheduledExecutorService;
- }
-
- public static boolean runRepetitiveDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
- Map<byte[], byte[]> jobParameters, long duration, EntityId entityId, ITxnIdFactory txnIdFactory)
- throws Exception {
- long executionMilliseconds = runDeployedJobSpec(distributedId, hcc, jobParameters, entityId, txnIdFactory);
- if (executionMilliseconds > duration && LOGGER.isErrorEnabled()) {
- LOGGER.log(Level.ERROR,
- "Periodic job for " + entityId.getExtensionName() + " " + entityId.getDataverse() + "."
- + entityId.getEntityName() + " was unable to meet the required period of " + duration
- + " milliseconds. Actually took " + executionMilliseconds + " execution will shutdown"
- + new Date());
- return false;
- }
- return true;
- }
-
- public synchronized static long runDeployedJobSpec(DeployedJobSpecId distributedId, IHyracksClientConnection hcc,
- Map<byte[], byte[]> jobParameters, EntityId entityId, ITxnIdFactory txnIdFactory) throws Exception {
- JobId jobId;
- long startTime = Instant.now().toEpochMilli();
-
- //Add the Asterix Transaction Id to the map
- jobParameters.put(TRANSACTION_ID_PARAMETER_NAME, String.valueOf(txnIdFactory.create().getId()).getBytes());
- jobId = hcc.startJob(distributedId, jobParameters);
-
- hcc.waitForCompletion(jobId);
- long executionMilliseconds = Instant.now().toEpochMilli() - startTime;
-
- LOGGER.log(Level.INFO,
- "Deployed Job execution completed for " + entityId.getExtensionName() + " " + entityId.getDataverse()
- + "." + entityId.getEntityName() + ". Took " + executionMilliseconds + " milliseconds ");
-
- return executionMilliseconds;
-
- }
-
- @Override
- public String toString() {
- return "DeployedJobSpecService";
- }
-
-}
diff --git a/asterixdb/asterix-algebra/pom.xml b/asterixdb/asterix-algebra/pom.xml
index 6a6bb8d..f3fdfc8 100644
--- a/asterixdb/asterix-algebra/pom.xml
+++ b/asterixdb/asterix-algebra/pom.xml
@@ -238,5 +238,9 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ </dependency>
</dependencies>
</project>
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 beb47a8..39d33f0 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
@@ -47,6 +47,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -59,6 +60,7 @@
import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
/**
* Contributes the runtime operator for an unnest-map representing a BTree search.
@@ -120,13 +122,23 @@
MetadataProvider metadataProvider = (MetadataProvider) context.getMetadataProvider();
Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
+ ITupleFilterFactory tupleFilterFactory = null;
+ long outputLimit = -1;
+ if (unnestMap.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ UnnestMapOperator unnestMapOp = (UnnestMapOperator) unnestMap;
+ outputLimit = unnestMapOp.getOutputLimit();
+ if (unnestMapOp.getSelectCondition() != null) {
+ tupleFilterFactory = metadataProvider.createTupleFilterFactory(new IOperatorSchema[] { opSchema },
+ typeEnv, unnestMapOp.getSelectCondition().getValue(), context);
+ }
+ }
// By nature, LEFT_OUTER_UNNEST_MAP should generate null values for non-matching tuples.
boolean retainMissing = op.getOperatorTag() == LogicalOperatorTag.LEFT_OUTER_UNNEST_MAP;
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
builder.getJobSpec(), opSchema, typeEnv, context, jobGenParams.getRetainInput(), retainMissing, dataset,
jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
jobGenParams.isHighKeyInclusive(), propagateFilter, minFilterFieldIndexes, maxFilterFieldIndexes,
- unnestMap.getGenerateCallBackProceedResultVar());
+ tupleFilterFactory, outputLimit, unnestMap.getGenerateCallBackProceedResultVar());
builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
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 cd217ab..134c96f 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
@@ -63,6 +63,7 @@
import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
import org.apache.asterix.optimizer.rules.PushGroupByThroughProduct;
import org.apache.asterix.optimizer.rules.PushLimitIntoOrderByRule;
+import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
import org.apache.asterix.optimizer.rules.PushProperJoinThroughProduct;
import org.apache.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
@@ -350,6 +351,7 @@
// We are going to apply a constant folding rule again for this case.
physicalRewritesTopLevel.add(new ConstantFoldingRule(appCtx));
physicalRewritesTopLevel.add(new PushLimitIntoOrderByRule());
+ physicalRewritesTopLevel.add(new PushLimitIntoPrimarySearchRule());
physicalRewritesTopLevel.add(new IntroduceProjectsRule());
physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
physicalRewritesTopLevel.add(new IntroduceRapidFrameFlushProjectAssignRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
index ab359bf..444e5bc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
@@ -75,11 +75,13 @@
* @return true if it is; false otherwise.
*/
private boolean isPossibleBoolean(IAType type) {
- while (NonTaggedFormatUtil.isOptional(type)) {
- IAType actualType = ((AUnionType) type).getActualType();
+ IAType checkingType = type;
+ while (NonTaggedFormatUtil.isOptional(checkingType)) {
+ IAType actualType = ((AUnionType) checkingType).getActualType();
if (actualType.getTypeTag() == ATypeTag.BOOLEAN || actualType.getTypeTag() == ATypeTag.ANY) {
return true;
}
+ checkingType = actualType;
}
return false;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 29a8e77..7110dbb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -22,6 +22,7 @@
import java.io.DataInputStream;
import java.nio.ByteBuffer;
import java.util.List;
+import java.util.Map;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -37,6 +38,7 @@
import org.apache.asterix.formats.nontagged.TypeTraitProvider;
import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.IAObject;
import org.apache.asterix.om.constants.AsterixConstantValue;
import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -78,6 +80,7 @@
import org.apache.hyracks.data.std.primitive.VoidPointable;
import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
public class ConstantFoldingRule implements IAlgebraicRewriteRule {
@@ -95,6 +98,9 @@
BuiltinFunctions.META_KEY, BuiltinFunctions.RECORD_CONCAT, BuiltinFunctions.RECORD_CONCAT_STRICT,
BuiltinFunctions.TO_ATOMIC, BuiltinFunctions.TO_ARRAY);
+ private static final Map<FunctionIdentifier, IAObject> FUNC_ID_TO_CONSTANT = ImmutableMap
+ .of(BuiltinFunctions.NUMERIC_E, new ADouble(Math.E), BuiltinFunctions.NUMERIC_PI, new ADouble(Math.PI));
+
/**
* Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method.
*/
@@ -220,6 +226,10 @@
return new Pair<>(changed, expr);
}
}
+ IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
+ if (c != null) {
+ return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(c)));
+ }
IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
_emptyTypeEnv, _emptySchemas, jobGenCtx);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
index c3e036e..14c3d87 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceAutogenerateIDRule.java
@@ -66,11 +66,14 @@
throws AlgebricksException {
// match: commit OR distribute-result OR SINK - ... followed by:
- // [insert to internal dataset with autogenerated id] - assign - project
- // produce: insert - assign - assign* - project
+ // [insert to internal dataset with autogenerated id] - assign - assign? - project
+ // produce: insert - assign - assign? - assign* - project
// **
- // OR [insert to internal dataset with autogenerated id] - assign - [datasource scan]
- // produce insert - assign - assign* - datasource scan
+ // OR [insert to internal dataset with autogenerated id] - assign - assign? - [datasource scan]
+ // produce: insert - assign - assign? - assign* - datasource scan
+ // **
+ // where assign* is the newly added assign that adds the autogenerated id
+ // and assign? is an assign that may exist when a filter is used
AbstractLogicalOperator currentOp = (AbstractLogicalOperator) opRef.getValue();
if (currentOp.getOperatorTag() == LogicalOperatorTag.DELEGATE_OPERATOR) {
@@ -120,15 +123,28 @@
AssignOperator assignOp = (AssignOperator) parentOp;
LogicalVariable inputRecord;
- //TODO: bug here. will not work for internal datasets with filters since the pattern becomes
- //[project-assign-assign-insert]
+ boolean hasFilter = false;
AbstractLogicalOperator grandparentOp = (AbstractLogicalOperator) parentOp.getInputs().get(0).getValue();
+ AbstractLogicalOperator newAssignParentOp = grandparentOp;
+ AbstractLogicalOperator newAssignChildOp = assignOp;
if (grandparentOp.getOperatorTag() == LogicalOperatorTag.PROJECT) {
ProjectOperator projectOp = (ProjectOperator) grandparentOp;
inputRecord = projectOp.getVariables().get(0);
} else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
DataSourceScanOperator dssOp = (DataSourceScanOperator) grandparentOp;
inputRecord = dssOp.getVariables().get(0);
+ } else if (grandparentOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+ AbstractLogicalOperator greatgrandparentOp =
+ (AbstractLogicalOperator) grandparentOp.getInputs().get(0).getValue();
+ if (greatgrandparentOp.getOperatorTag() != LogicalOperatorTag.PROJECT) {
+ return false;
+ }
+ //filter case
+ ProjectOperator projectOp = (ProjectOperator) greatgrandparentOp;
+ inputRecord = projectOp.getVariables().get(0);
+ newAssignParentOp = greatgrandparentOp;
+ newAssignChildOp = grandparentOp;
+ hasFilter = true;
} else {
return false;
}
@@ -142,13 +158,19 @@
LogicalVariable v = context.newVar();
AssignOperator newAssign = new AssignOperator(v, new MutableObject<ILogicalExpression>(nonNullMergedRec));
- newAssign.getInputs().add(new MutableObject<ILogicalOperator>(grandparentOp));
- assignOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+ newAssign.getInputs().add(new MutableObject<ILogicalOperator>(newAssignParentOp));
+ newAssignChildOp.getInputs().set(0, new MutableObject<ILogicalOperator>(newAssign));
+ if (hasFilter) {
+ VariableUtilities.substituteVariables(newAssignChildOp, inputRecord, v, context);
+ }
VariableUtilities.substituteVariables(assignOp, inputRecord, v, context);
VariableUtilities.substituteVariables(insertOp, inputRecord, v, context);
context.computeAndSetTypeEnvironmentForOperator(newAssign);
+ if (hasFilter) {
+ context.computeAndSetTypeEnvironmentForOperator(newAssignChildOp);
+ }
context.computeAndSetTypeEnvironmentForOperator(assignOp);
- context.computeAndSetTypeEnvironmentForOperator(insertOp);;
+ context.computeAndSetTypeEnvironmentForOperator(insertOp);
for (AbstractLogicalOperator op : opStack) {
VariableUtilities.substituteVariables(op, inputRecord, v, context);
context.computeAndSetTypeEnvironmentForOperator(op);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
new file mode 100644
index 0000000..921d231
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.asterix.metadata.declared.DataSource;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+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.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.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pattern:
+ * SCAN or UNNEST_MAP -> (SELECT)? -> (EXCHANGE)? -> LIMIT
+ * We push both SELECT condition and LIMIT to SCAN or UNNEST_MAP
+ *
+ */
+public class PushLimitIntoPrimarySearchRule implements IAlgebraicRewriteRule {
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
+ ILogicalOperator op = opRef.getValue();
+ if (op.getOperatorTag() != LogicalOperatorTag.LIMIT) {
+ return false;
+ }
+ if (context.checkIfInDontApplySet(this, op)) {
+ return false;
+ }
+ context.addToDontApplySet(this, op);
+
+ Long outputLimit = getOutputLimit((LimitOperator) op);
+ if (outputLimit == null) {
+ // we cannot push if limit is not constant
+ return false;
+ }
+
+ Mutable<ILogicalOperator> childOp = op.getInputs().get(0);
+ if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
+ childOp = childOp.getValue().getInputs().get(0);
+ }
+ boolean changed = false;
+ if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.SELECT) {
+ changed = rewriteSelect(childOp, outputLimit);
+ } else {
+ changed = setLimitForScanOrUnnestMap(childOp.getValue(), outputLimit);
+ }
+ if (changed) {
+ OperatorPropertiesUtil.typeOpRec(opRef, context);
+ }
+ return changed;
+ }
+
+ private Long getOutputLimit(LimitOperator limit) {
+ if (limit.getMaxObjects().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return null;
+ }
+ long outputLimit = AccessMethodUtils.getInt64Constant(limit.getMaxObjects());
+ if (limit.getOffset() != null && limit.getOffset().getValue() != null) {
+ if (limit.getOffset().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ return null;
+ }
+ outputLimit += AccessMethodUtils.getInt64Constant(limit.getOffset());
+ }
+ return outputLimit;
+ }
+
+ private boolean rewriteSelect(Mutable<ILogicalOperator> op, long outputLimit) throws AlgebricksException {
+ SelectOperator select = (SelectOperator) op.getValue();
+ Set<LogicalVariable> selectedVariables = new HashSet<>();
+ select.getCondition().getValue().getUsedVariables(selectedVariables);
+ ILogicalOperator child = select.getInputs().get(0).getValue();
+ boolean changed = false;
+ if (child.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) child;
+ if (isScanPushable(scan, selectedVariables)) {
+ scan.setSelectCondition(select.getCondition());
+ scan.setOutputLimit(outputLimit);
+ changed = true;
+ }
+ } else if (child.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ UnnestMapOperator unnestMap = (UnnestMapOperator) child;
+ if (isUnnestMapPushable(unnestMap, selectedVariables)) {
+ unnestMap.setSelectCondition(select.getCondition());
+ unnestMap.setOutputLimit(outputLimit);
+ changed = true;
+ }
+ }
+ if (changed) {
+ // SELECT is not needed
+ op.setValue(child);
+ }
+ return changed;
+ }
+
+ private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, long outputLimit) throws AlgebricksException {
+ if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+ DataSourceScanOperator scan = (DataSourceScanOperator) op;
+ if (isScanPushable(scan, Collections.emptySet())) {
+ scan.setOutputLimit(outputLimit);
+ return true;
+ }
+ } else if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+ UnnestMapOperator unnestMap = (UnnestMapOperator) op;
+ if (isUnnestMapPushable(unnestMap, Collections.emptySet())) {
+ unnestMap.setOutputLimit(outputLimit);
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean isUnnestMapPushable(UnnestMapOperator op, Set<LogicalVariable> selectedVariables) {
+ if (op.getOutputLimit() >= 0) {
+ // already pushed
+ return false;
+ }
+ ILogicalExpression unnestExpr = op.getExpressionRef().getValue();
+ if (op.propagatesInput() || unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+ if (!f.getFunctionIdentifier().equals(BuiltinFunctions.INDEX_SEARCH)) {
+ return false;
+ }
+ AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+ jobGenParams.readFromFuncArgs(f.getArguments());
+ if (!jobGenParams.isPrimaryIndex()) {
+ return false;
+ }
+ if (!op.getScanVariables().containsAll(selectedVariables)) {
+ return false;
+ }
+ return true;
+ }
+
+ private boolean isScanPushable(DataSourceScanOperator op, Set<LogicalVariable> selectedVariables) {
+ if (op.getOutputLimit() >= 0) {
+ return false;
+ }
+ if (!op.getInputs().isEmpty()
+ && op.getInputs().get(0).getValue().getOperatorTag() != LogicalOperatorTag.EMPTYTUPLESOURCE) {
+ return false;
+ }
+ if (((DataSource) op.getDataSource()).getDatasourceType() != DataSource.Type.INTERNAL_DATASET) {
+ return false;
+ }
+ if (!op.getScanVariables().containsAll(selectedVariables)) {
+ return false;
+ }
+ return true;
+ }
+
+}
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 4f9b4df..1a4f1c0 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
@@ -96,7 +96,7 @@
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_MOD);
+ BuiltinFunctions.NUMERIC_DIVIDE, BuiltinFunctions.NUMERIC_DIV, BuiltinFunctions.NUMERIC_MOD);
public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
new file mode 100644
index 0000000..d77164c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlans.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.translator;
+
+import java.io.Serializable;
+
+public class ExecutionPlans implements Serializable {
+
+ private String expressionTree;
+ private String rewrittenExpressionTree;
+ private String logicalPlan;
+ private String optimizedLogicalPlan;
+ private String job;
+
+ public String getExpressionTree() {
+ return expressionTree;
+ }
+
+ public void setExpressionTree(String expressionTree) {
+ this.expressionTree = expressionTree;
+ }
+
+ public String getRewrittenExpressionTree() {
+ return rewrittenExpressionTree;
+ }
+
+ public void setRewrittenExpressionTree(String rewrittenExpressionTree) {
+ this.rewrittenExpressionTree = rewrittenExpressionTree;
+ }
+
+ public String getLogicalPlan() {
+ return logicalPlan;
+ }
+
+ public void setLogicalPlan(String logicalPlan) {
+ this.logicalPlan = logicalPlan;
+ }
+
+ public String getOptimizedLogicalPlan() {
+ return optimizedLogicalPlan;
+ }
+
+ public void setOptimizedLogicalPlan(String optimizedLogicalPlan) {
+ this.optimizedLogicalPlan = optimizedLogicalPlan;
+ }
+
+ public String getJob() {
+ return job;
+ }
+
+ public void setJob(String job) {
+ this.job = job;
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java
new file mode 100644
index 0000000..88e8255
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansHtmlPrintUtil.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.translator;
+
+import java.io.PrintWriter;
+
+public class ExecutionPlansHtmlPrintUtil {
+
+ private static final String LOGICAL_PLAN_LBL = "Logical plan";
+ private static final String EXPRESSION_TREE_LBL = "Expression tree";
+ private static final String REWRITTEN_EXPRESSION_TREE_LBL = "Rewritten expression tree";
+ private static final String OPTIMIZED_LOGICAL_PLAN_LBL = "Optimized logical plan";
+ private static final String JOB_LBL = "Job";
+
+ private ExecutionPlansHtmlPrintUtil() {
+ }
+
+ public static void print(PrintWriter output, ExecutionPlans plans) {
+ printNonNull(output, EXPRESSION_TREE_LBL, plans.getExpressionTree());
+ printNonNull(output, REWRITTEN_EXPRESSION_TREE_LBL, plans.getRewrittenExpressionTree());
+ printNonNull(output, LOGICAL_PLAN_LBL, plans.getLogicalPlan());
+ printNonNull(output, OPTIMIZED_LOGICAL_PLAN_LBL, plans.getOptimizedLogicalPlan());
+ printNonNull(output, JOB_LBL, plans.getJob());
+ }
+
+ private static void printNonNull(PrintWriter output, String lbl, String value) {
+ if (value != null) {
+ printFieldPrefix(output, lbl);
+ output.print(value);
+ printFieldPostfix(output);
+ }
+ }
+
+ private static void printFieldPrefix(PrintWriter output, String lbl) {
+ output.println();
+ output.println("<h4>" + lbl + ":</h4>");
+ switch (lbl) {
+ case LOGICAL_PLAN_LBL:
+ output.println("<pre class=query-plan>");
+ break;
+ case OPTIMIZED_LOGICAL_PLAN_LBL:
+ output.println("<pre class=query-optimized-plan>");
+ break;
+ default:
+ output.println("<pre>");
+ break;
+ }
+ }
+
+ private static void printFieldPostfix(PrintWriter output) {
+ output.println("</pre>");
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
new file mode 100644
index 0000000..5c47ca2
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ExecutionPlansJsonPrintUtil.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.translator;
+
+import static org.apache.asterix.translator.SessionConfig.PlanFormat.STRING;
+
+import org.apache.hyracks.util.JSONUtil;
+
+public class ExecutionPlansJsonPrintUtil {
+
+ private static final String LOGICAL_PLAN_LBL = "logicalPlan";
+ private static final String EXPRESSION_TREE_LBL = "expressionTree";
+ private static final String REWRITTEN_EXPRESSION_TREE_LBL = "rewrittenExpressionTree";
+ private static final String OPTIMIZED_LOGICAL_PLAN_LBL = "optimizedLogicalPlan";
+ private static final String JOB_LBL = "job";
+
+ private ExecutionPlansJsonPrintUtil() {
+ }
+
+ public static String asJson(ExecutionPlans plans, SessionConfig.PlanFormat format) {
+ final StringBuilder output = new StringBuilder();
+ appendOutputPrefix(output);
+ // TODO only string is currently supported for expression trees
+ appendNonNull(output, EXPRESSION_TREE_LBL, plans.getExpressionTree(), STRING);
+ appendNonNull(output, REWRITTEN_EXPRESSION_TREE_LBL, plans.getRewrittenExpressionTree(), STRING);
+ appendNonNull(output, LOGICAL_PLAN_LBL, plans.getLogicalPlan(), format);
+ appendNonNull(output, OPTIMIZED_LOGICAL_PLAN_LBL, plans.getOptimizedLogicalPlan(), format);
+ appendNonNull(output, JOB_LBL, plans.getJob(), format);
+ appendOutputPostfix(output);
+ return output.toString();
+ }
+
+ private static void appendNonNull(StringBuilder builder, String lbl, String value,
+ SessionConfig.PlanFormat format) {
+ if (value != null) {
+ printFieldPrefix(builder, lbl);
+ switch (format) {
+ case JSON:
+ builder.append(value);
+ break;
+ case STRING:
+ JSONUtil.quoteAndEscape(builder, value);
+ break;
+ default:
+ throw new IllegalStateException("Unrecognized plan format: " + format);
+ }
+ printFieldPostfix(builder);
+ }
+ }
+
+ private static void appendOutputPrefix(StringBuilder builder) {
+ builder.append("{");
+ }
+
+ private static void printFieldPrefix(StringBuilder builder, String lbl) {
+ builder.append("\"" + lbl + "\": ");
+ }
+
+ private static void printFieldPostfix(StringBuilder builder) {
+ builder.append(",");
+ }
+
+ private static void appendOutputPostfix(StringBuilder builder) {
+ // remove extra comma if needed
+ if (builder.length() > 1) {
+ builder.deleteCharAt(builder.length() - 1);
+ }
+ builder.append("}");
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index d76c421..0ff877b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -142,4 +142,11 @@
*/
String getActiveDataverseName(String dataverse);
+ /**
+ * Gets the execution plans that are generated during query compilation
+ *
+ * @return the executions plans
+ */
+ ExecutionPlans getExecutionPlans();
+
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 6ab76fa..9a6870d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -42,7 +42,6 @@
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Expression.Kind;
import org.apache.asterix.lang.common.base.ILangExpression;
-import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.clause.LimitClause;
@@ -1261,17 +1260,17 @@
case MUL:
fid = BuiltinFunctions.NUMERIC_MULTIPLY;
break;
- case DIV:
+ case DIVIDE:
fid = BuiltinFunctions.NUMERIC_DIVIDE;
break;
+ case DIV:
+ fid = BuiltinFunctions.NUMERIC_DIV;
+ break;
case MOD:
fid = BuiltinFunctions.NUMERIC_MOD;
break;
- case IDIV:
- fid = BuiltinFunctions.NUMERIC_IDIV;
- break;
case CARET:
- fid = BuiltinFunctions.CARET;
+ fid = BuiltinFunctions.NUMERIC_POWER;
break;
case AND:
fid = AlgebricksBuiltinFunctions.AND;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
index cb6d8e5..89619e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionConfig.java
@@ -63,7 +63,7 @@
if (fmtString != null) {
String format = ("JSON".equalsIgnoreCase(fmtString) || "CLEAN_JSON".equalsIgnoreCase(fmtString))
? "JSON" : fmtString;
- return PlanFormat.valueOf(format);
+ return PlanFormat.valueOf(format.toUpperCase());
}
} catch (IllegalArgumentException e) {
logger.log(Level.INFO, fmtString + ": unsupported " + label + ", using " + defaultFmt + "instead", e);
@@ -129,7 +129,7 @@
// Output format.
private final OutputFormat fmt;
- private final PlanFormat lpfmt;
+ private final PlanFormat planFormat;
// Standard execution flags.
private final boolean executeQuery;
@@ -143,8 +143,8 @@
this(fmt, PlanFormat.STRING);
}
- public SessionConfig(OutputFormat fmt, PlanFormat lpfmt) {
- this(fmt, true, true, true, lpfmt);
+ public SessionConfig(OutputFormat fmt, PlanFormat planFormat) {
+ this(fmt, true, true, true, planFormat);
}
/**
@@ -168,13 +168,13 @@
}
public SessionConfig(OutputFormat fmt, boolean optimize, boolean executeQuery, boolean generateJobSpec,
- PlanFormat lpfmt) {
+ PlanFormat planFormat) {
this.fmt = fmt;
this.optimize = optimize;
this.executeQuery = executeQuery;
this.generateJobSpec = generateJobSpec;
this.flags = new HashMap<>();
- this.lpfmt = lpfmt;
+ this.planFormat = planFormat;
}
/**
@@ -187,8 +187,8 @@
/**
* Retrieve the PlanFormat for this execution.
*/
- public PlanFormat getLpfmt() {
- return this.lpfmt;
+ public PlanFormat getPlanFormat() {
+ return this.planFormat;
}
/**
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 4b4b2b0..d5ebc6e 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
@@ -131,11 +131,19 @@
} else {
partitioningExprTypes =
KeyFieldTypeUtil.getKeyTypes(recType, metaRecType, partitioningExprs, keySourceIndicators);
- for (int fidx = 0; fidx < partitioningExprTypes.size(); ++fidx) {
- IAType fieldType = partitioningExprTypes.get(fidx);
+ for (int i = 0; i < partitioningExprs.size(); i++) {
+ List<String> partitioningExpr = partitioningExprs.get(i);
+ IAType fieldType = partitioningExprTypes.get(i);
if (fieldType == null) {
throw new CompilationException(ErrorCode.COMPILATION_FIELD_NOT_FOUND,
- RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+ RecordUtil.toFullyQualifiedName(partitioningExpr));
+ }
+ boolean nullable = KeyFieldTypeUtil.chooseSource(keySourceIndicators, i, recType, metaRecType)
+ .isSubFieldNullable(partitioningExpr);
+ if (nullable) {
+ // key field is nullable
+ throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE,
+ RecordUtil.toFullyQualifiedName(partitioningExpr));
}
switch (fieldType.getTypeTag()) {
case TINYINT:
@@ -155,7 +163,7 @@
break;
case UNION:
throw new CompilationException(ErrorCode.COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE,
- RecordUtil.toFullyQualifiedName(partitioningExprs.get(fidx)));
+ RecordUtil.toFullyQualifiedName(partitioningExpr));
default:
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE,
fieldType.getTypeTag());
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 020bc6c..a39c366 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -153,6 +153,7 @@
<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>
<ignoredUnusedDeclaredDependency>org.apache.asterix:asterix-external-data:zip:*</ignoredUnusedDeclaredDependency>
@@ -648,5 +649,11 @@
<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>
+ </dependency>
</dependencies>
</project>
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 ad715a4..537625d 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
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.io.PrintWriter;
+import java.io.StringWriter;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@@ -67,6 +68,7 @@
import org.apache.asterix.optimizer.rules.am.AbstractIntroduceAccessMethodRule;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.ExecutionPlans;
import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.asterix.translator.SessionConfig;
import org.apache.asterix.translator.SessionOutput;
@@ -120,8 +122,7 @@
private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
// one for query, two for intermediate results, one for final result, and one for reading an inverted list
private static final int MIN_FRAME_LIMIT_FOR_TEXTSEARCH = 5;
- private static final String LPLAN = "Logical plan";
- private static final String OPLAN = "Optimized logical plan";
+ private static final ObjectWriter OBJECT_WRITER = new ObjectMapper().writerWithDefaultPrettyPrinter();
// A white list of supported configurable parameters.
private static final Set<String> CONFIGURABLE_PARAMETER_NAMES =
@@ -137,12 +138,14 @@
private final IAstPrintVisitorFactory astPrintVisitorFactory;
private final ILangExpressionToPlanTranslatorFactory translatorFactory;
private final IRuleSetFactory ruleSetFactory;
+ private final ExecutionPlans executionPlans;
public APIFramework(ILangCompilationProvider compilationProvider) {
this.rewriterFactory = compilationProvider.getRewriterFactory();
this.astPrintVisitorFactory = compilationProvider.getAstPrintVisitorFactory();
this.translatorFactory = compilationProvider.getExpressionToPlanTranslatorFactory();
this.ruleSetFactory = compilationProvider.getRuleSetFactory();
+ executionPlans = new ExecutionPlans();
}
private static class OptimizationContextFactory implements IOptimizationContextFactory {
@@ -165,27 +168,6 @@
}
}
- private void printPlanPrefix(SessionOutput output, String planName) {
- if (output.config().is(SessionConfig.FORMAT_HTML)) {
- output.out().println("<h4>" + planName + ":</h4>");
- if (LPLAN.equalsIgnoreCase(planName)) {
- output.out().println("<pre class = query-plan>");
- } else if (OPLAN.equalsIgnoreCase(planName)) {
- output.out().println("<pre class = query-optimized-plan>");
- } else {
- output.out().println("<pre>");
- }
- } else {
- output.out().println("----------" + planName + ":");
- }
- }
-
- private void printPlanPostfix(SessionOutput output) {
- if (output.config().is(SessionConfig.FORMAT_HTML)) {
- output.out().println("</pre>");
- }
- }
-
public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
MetadataProvider metadataProvider, IReturningStatement q, SessionOutput output, boolean inlineUdfs)
throws CompilationException {
@@ -194,10 +176,7 @@
}
SessionConfig conf = output.config();
if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_EXPR_TREE)) {
- output.out().println();
- printPlanPrefix(output, "Expression tree");
- q.accept(astPrintVisitorFactory.createLangVisitor(output.out()), 0);
- printPlanPostfix(output);
+ generateExpressionTree(q);
}
IQueryRewriter rw = rewriterFactory.createQueryRewriter();
rw.rewrite(declaredFunctions, q, metadataProvider, new LangRewritingContext(q.getVarCounter()), inlineUdfs);
@@ -213,14 +192,9 @@
final boolean isLoad = statement != null && statement.getKind() == Statement.Kind.LOAD;
SessionConfig conf = output.config();
- if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
- output.out().println();
-
- printPlanPrefix(output, "Rewritten expression tree");
- if (isQuery) {
- query.accept(astPrintVisitorFactory.createLangVisitor(output.out()), 0);
- }
- printPlanPostfix(output);
+ if (isQuery && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+ && conf.is(SessionConfig.OOB_REWRITTEN_EXPR_TREE)) {
+ generateRewrittenExpressionTree(query);
}
final TxnId txnId = metadataProvider.getTxnIdFactory().create();
@@ -230,14 +204,9 @@
ILogicalPlan plan = isLoad ? t.translateLoad(statement) : t.translate(query, outputDatasetName, statement);
- if (!conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS) && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
- output.out().println();
-
- printPlanPrefix(output, "Logical plan");
- if (isQuery || isLoad) {
- PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(output.config().getLpfmt(), output.out()), 0);
- }
- printPlanPostfix(output);
+ if ((isQuery || isLoad) && !conf.is(SessionConfig.FORMAT_ONLY_PHYSICAL_OPS)
+ && conf.is(SessionConfig.OOB_LOGICAL_PLAN)) {
+ generateLogicalPlan(plan, output.config().getPlanFormat());
}
CompilerProperties compilerProperties = metadataProvider.getApplicationContext().getCompilerProperties();
Map<String, String> querySpecificConfig = validateConfig(metadataProvider.getConfig());
@@ -273,12 +242,9 @@
AlgebricksAppendable buffer = new AlgebricksAppendable(output.out());
PlanPrettyPrinter.printPhysicalOps(plan, buffer, 0);
} else {
- printPlanPrefix(output, "Optimized logical plan");
if (isQuery || isLoad) {
- PlanPrettyPrinter.printPlan(plan,
- getPrettyPrintVisitor(output.config().getLpfmt(), output.out()), 0);
+ generateOptimizedLogicalPlan(plan, output.config().getPlanFormat());
}
- printPlanPostfix(output);
}
}
}
@@ -327,8 +293,9 @@
ResourceUtils.getRequiredCapacity(plan, jobLocations, physOptConf);
spec.setRequiredClusterCapacity(jobRequiredCapacity);
}
-
- printJobSpec(query, spec, conf, output);
+ if (isQuery && conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
+ generateJob(spec);
+ }
return spec;
}
@@ -373,23 +340,6 @@
}
}
- protected void printJobSpec(Query rwQ, JobSpecification spec, SessionConfig conf, SessionOutput output)
- throws AlgebricksException {
- if (conf.is(SessionConfig.OOB_HYRACKS_JOB)) {
- printPlanPrefix(output, "Hyracks job");
- if (rwQ != null) {
- try {
- final ObjectWriter objectWriter = new ObjectMapper().writerWithDefaultPrettyPrinter();
- output.out().println(objectWriter.writeValueAsString(spec.toJSON()));
- } catch (IOException e) {
- throw new AlgebricksException(e);
- }
- output.out().println(spec.getUserConstraints());
- }
- printPlanPostfix(output);
- }
- }
-
private AbstractLogicalOperatorPrettyPrintVisitor getPrettyPrintVisitor(SessionConfig.PlanFormat planFormat,
PrintWriter out) {
return planFormat.equals(SessionConfig.PlanFormat.JSON) ? new LogicalOperatorPrettyPrintVisitorJson(out)
@@ -429,6 +379,10 @@
}
}
+ public ExecutionPlans getExecutionPlans() {
+ return executionPlans;
+ }
+
// Chooses the location constraints, i.e., whether to use storage parallelism or use a user-sepcified number
// of cores.
private static AlgebricksAbsolutePartitionConstraint chooseLocations(IClusterInfoCollector clusterInfoCollector,
@@ -524,6 +478,49 @@
return config;
}
+ private void generateExpressionTree(IReturningStatement statement) throws CompilationException {
+ final StringWriter stringWriter = new StringWriter();
+ try (PrintWriter writer = new PrintWriter(stringWriter)) {
+ statement.accept(astPrintVisitorFactory.createLangVisitor(writer), 0);
+ executionPlans.setExpressionTree(stringWriter.toString());
+ }
+ }
+
+ private void generateRewrittenExpressionTree(IReturningStatement statement) throws CompilationException {
+ final StringWriter stringWriter = new StringWriter();
+ try (PrintWriter writer = new PrintWriter(stringWriter)) {
+ statement.accept(astPrintVisitorFactory.createLangVisitor(writer), 0);
+ executionPlans.setRewrittenExpressionTree(stringWriter.toString());
+ }
+ }
+
+ private void generateLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format) throws AlgebricksException {
+ final StringWriter stringWriter = new StringWriter();
+ try (PrintWriter writer = new PrintWriter(stringWriter)) {
+ PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(format, writer), 0);
+ executionPlans.setLogicalPlan(stringWriter.toString());
+ }
+ }
+
+ private void generateOptimizedLogicalPlan(ILogicalPlan plan, SessionConfig.PlanFormat format)
+ throws AlgebricksException {
+ final StringWriter stringWriter = new StringWriter();
+ try (PrintWriter writer = new PrintWriter(stringWriter)) {
+ PlanPrettyPrinter.printPlan(plan, getPrettyPrintVisitor(format, writer), 0);
+ executionPlans.setOptimizedLogicalPlan(stringWriter.toString());
+ }
+ }
+
+ private void generateJob(JobSpecification spec) {
+ final StringWriter stringWriter = new StringWriter();
+ try (PrintWriter writer = new PrintWriter(stringWriter)) {
+ writer.println(OBJECT_WRITER.writeValueAsString(spec.toJSON()));
+ executionPlans.setJob(stringWriter.toString());
+ } catch (IOException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
public static AlgebricksAbsolutePartitionConstraint getJobLocations(JobSpecification spec,
INodeJobTracker jobTracker, AlgebricksAbsolutePartitionConstraint clusterLocations) {
final Set<String> jobParticipatingNodes = jobTracker.getJobParticipatingNodes(spec);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index b8c737d..bd096dd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -51,7 +51,8 @@
RESULTS("results"),
HANDLE("handle"),
ERRORS("errors"),
- METRICS("metrics");
+ METRICS("metrics"),
+ PLANS("plans");
private final String str;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
deleted file mode 100644
index 4c1dbf7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/DdlApiServlet.java
+++ /dev/null
@@ -1,55 +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.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class DdlApiServlet extends RestApiServlet {
- private static final byte ALLOWED_CATEGORIES =
- Statement.Category.QUERY | Statement.Category.UPDATE | Statement.Category.DDL;
-
- public DdlApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
- ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
- IStorageComponentProvider componentProvider) {
- super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
- }
-
- @Override
- protected String getQueryParameter(IServletRequest request) {
- return request.getParameter("ddl");
- }
-
- @Override
- protected byte getAllowedCategories() {
- return ALLOWED_CATEGORIES;
- }
-
- @Override
- protected String getErrorMessage() {
- return "Invalid statement: Non-DDL statement %s to the DDL API.";
- }
-
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
deleted file mode 100644
index 0a461c7..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/FullApiServlet.java
+++ /dev/null
@@ -1,56 +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.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class FullApiServlet extends RestApiServlet {
-
- private static final String AQL_STMT_PARAM_NAME = "aql";
- private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE
- | Statement.Category.DDL | Statement.Category.PROCEDURE;
-
- public FullApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
- ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
- IStorageComponentProvider componentProvider) {
- super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
- }
-
- @Override
- protected byte getAllowedCategories() {
- return ALLOWED_CATEGORIES;
- }
-
- @Override
- protected String getErrorMessage() {
- throw new IllegalStateException();
- }
-
- @Override
- protected String getQueryParameter(IServletRequest request) {
- return request.getParameter(AQL_STMT_PARAM_NAME);
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index da621d2..5ad451f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -23,6 +23,7 @@
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
import org.apache.asterix.app.message.CancelQueryRequest;
import org.apache.asterix.app.message.CancelQueryResponse;
import org.apache.asterix.common.messaging.api.INCMessageBroker;
@@ -54,7 +55,7 @@
@Override
protected void delete(IServletRequest request, IServletResponse response) {
// gets the parameter client_context_id from the request.
- String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
+ String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
if (clientContextId == null) {
response.setStatus(HttpResponseStatus.BAD_REQUEST);
return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index a420efc..1713ca5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -132,6 +132,7 @@
} else {
sessionOutput.out().append(responseMsg.getResult());
}
+ printExecutionPlans(sessionOutput, responseMsg.getExecutionPlans());
}
private void cancelQuery(INCMessageBroker messageBroker, String nodeId, String clientContextID, Exception exception,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
deleted file mode 100644
index 3e692d3..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryApiServlet.java
+++ /dev/null
@@ -1,53 +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.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class QueryApiServlet extends RestApiServlet {
- private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY;
-
- public QueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
- ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
- IStorageComponentProvider componentProvider) {
- super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
- }
-
- @Override
- protected String getQueryParameter(IServletRequest request) {
- return request.getParameter("query");
- }
-
- @Override
- protected byte getAllowedCategories() {
- return ALLOWED_CATEGORIES;
- }
-
- @Override
- protected String getErrorMessage() {
- return "Invalid statement: Non-query statement %s to the query API.";
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
index f8655ad..a8b3aef 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.util.concurrent.ConcurrentMap;
+import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
import org.apache.asterix.translator.IStatementExecutorContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.job.JobId;
@@ -38,7 +39,6 @@
*/
public class QueryCancellationServlet extends AbstractServlet {
private static final Logger LOGGER = LogManager.getLogger();
- protected static final String CLIENT_CONTEXT_ID = "client_context_id";
public QueryCancellationServlet(ConcurrentMap<String, Object> ctx, String... paths) {
super(ctx, paths);
@@ -46,7 +46,7 @@
@Override
protected void delete(IServletRequest request, IServletResponse response) throws IOException {
- String clientContextId = request.getParameter(CLIENT_CONTEXT_ID);
+ String clientContextId = request.getParameter(Parameter.CLIENT_ID.str());
if (clientContextId == null) {
response.setStatus(HttpResponseStatus.BAD_REQUEST);
return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 56359e3..3d0858c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -45,6 +45,8 @@
import org.apache.asterix.lang.common.base.IParser;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
import org.apache.asterix.translator.IRequestParameters;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
@@ -73,6 +75,7 @@
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
+
import io.netty.handler.codec.http.HttpResponseStatus;
public class QueryServiceServlet extends AbstractQueryApiServlet {
@@ -141,7 +144,13 @@
MODE("mode"),
TIMEOUT("timeout"),
PLAN_FORMAT("plan-format"),
- MAX_RESULT_READS("max-result-reads");
+ MAX_RESULT_READS("max-result-reads"),
+ EXPRESSION_TREE("expression-tree"),
+ REWRITTEN_EXPRESSION_TREE("rewritten-expression-tree"),
+ LOGICAL_PLAN("logical-plan"),
+ OPTIMIZED_LOGICAL_PLAN("optimized-logical-plan"),
+ JOB("job"),
+ SIGNATURE("signature");
private final String str;
@@ -188,7 +197,7 @@
}
}
- static class RequestParameters {
+ protected static class RequestParameters {
String host;
String path;
String statement;
@@ -198,6 +207,13 @@
String clientContextID;
String mode;
String maxResultReads;
+ String planFormat;
+ boolean expressionTree;
+ boolean rewrittenExpressionTree;
+ boolean logicalPlan;
+ boolean optimizedLogicalPlan;
+ boolean job;
+ boolean signature;
@Override
public String toString() {
@@ -213,6 +229,13 @@
on.put("format", format);
on.put("timeout", timeout);
on.put("maxResultReads", maxResultReads);
+ on.put("planFormat", planFormat);
+ on.put("expressionTree", expressionTree);
+ on.put("rewrittenExpressionTree", rewrittenExpressionTree);
+ on.put("logicalPlan", logicalPlan);
+ on.put("optimizedLogicalPlan", optimizedLogicalPlan);
+ on.put("job", job);
+ on.put("signature", signature);
return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
} catch (JsonProcessingException e) { // NOSONAR
LOGGER.debug("unexpected exception marshalling {} instance to json", getClass(), e);
@@ -221,7 +244,7 @@
}
}
- static final class RequestExecutionState {
+ protected static final class RequestExecutionState {
private long execStart = -1;
private long execEnd = -1;
private ResultStatus resultStatus = ResultStatus.SUCCESS;
@@ -291,7 +314,7 @@
if (format.equals(HttpUtil.ContentType.APPLICATION_ADM)) {
return SessionConfig.OutputFormat.ADM;
}
- if (format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)) {
+ if (isJsonFormat(format)) {
return Boolean.parseBoolean(getParameterValue(format, Attribute.LOSSLESS.str()))
? SessionConfig.OutputFormat.LOSSLESS_JSON : SessionConfig.OutputFormat.CLEAN_JSON;
}
@@ -307,9 +330,15 @@
SessionOutput.ResultAppender appendStatus = ResultUtil.createResultStatusAppender();
SessionConfig.OutputFormat format = getFormat(param.format);
- //TODO:get the parameters from UI.Currently set to clean_json.
- SessionConfig sessionConfig = new SessionConfig(format);
+ final SessionConfig.PlanFormat planFormat =
+ SessionConfig.PlanFormat.get(param.planFormat, param.planFormat, SessionConfig.PlanFormat.JSON, LOGGER);
+ SessionConfig sessionConfig = new SessionConfig(format, planFormat);
sessionConfig.set(SessionConfig.FORMAT_WRAPPER_ARRAY, true);
+ sessionConfig.set(SessionConfig.OOB_EXPR_TREE, param.expressionTree);
+ sessionConfig.set(SessionConfig.OOB_REWRITTEN_EXPR_TREE, param.rewrittenExpressionTree);
+ sessionConfig.set(SessionConfig.OOB_LOGICAL_PLAN, param.logicalPlan);
+ sessionConfig.set(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN, param.optimizedLogicalPlan);
+ sessionConfig.set(SessionConfig.OOB_HYRACKS_JOB, param.job);
sessionConfig.set(SessionConfig.FORMAT_INDENT_JSON, param.pretty);
sessionConfig.set(SessionConfig.FORMAT_QUOTE_RECORD,
format != SessionConfig.OutputFormat.CLEAN_JSON && format != SessionConfig.OutputFormat.LOSSLESS_JSON);
@@ -324,8 +353,15 @@
}
}
- private static void printSignature(PrintWriter pw) {
- ResultUtil.printField(pw, ResultFields.SIGNATURE.str(), "*");
+ private static void printSignature(PrintWriter pw, RequestParameters param) {
+ if (param.signature) {
+ pw.print("\t\"");
+ pw.print(ResultFields.SIGNATURE.str());
+ pw.print("\": {\n");
+ pw.print("\t");
+ ResultUtil.printField(pw, "*", "*", false);
+ pw.print("\t},\n");
+ }
}
private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
@@ -391,6 +427,14 @@
param.clientContextID = getOptText(jsonRequest, Parameter.CLIENT_ID.str());
param.timeout = getOptText(jsonRequest, Parameter.TIMEOUT.str());
param.maxResultReads = getOptText(jsonRequest, Parameter.MAX_RESULT_READS.str());
+ param.planFormat = getOptText(jsonRequest, Parameter.PLAN_FORMAT.str());
+ param.expressionTree = getOptBoolean(jsonRequest, Parameter.EXPRESSION_TREE.str(), false);
+ param.rewrittenExpressionTree =
+ getOptBoolean(jsonRequest, Parameter.REWRITTEN_EXPRESSION_TREE.str(), false);
+ param.logicalPlan = getOptBoolean(jsonRequest, Parameter.LOGICAL_PLAN.str(), false);
+ param.optimizedLogicalPlan = getOptBoolean(jsonRequest, Parameter.OPTIMIZED_LOGICAL_PLAN.str(), false);
+ param.job = getOptBoolean(jsonRequest, Parameter.JOB.str(), false);
+ param.signature = getOptBoolean(jsonRequest, Parameter.SIGNATURE.str(), true);
} catch (JsonParseException | JsonMappingException e) {
// if the JSON parsing fails, the statement is empty and we get an empty statement error
GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, e.getMessage(), e);
@@ -406,6 +450,7 @@
param.clientContextID = request.getParameter(Parameter.CLIENT_ID.str());
param.timeout = request.getParameter(Parameter.TIMEOUT.str());
param.maxResultReads = request.getParameter(Parameter.MAX_RESULT_READS.str());
+ param.planFormat = request.getParameter(Parameter.PLAN_FORMAT.str());
}
return param;
}
@@ -473,7 +518,7 @@
sessionOutput.out().print("{\n");
printRequestId(sessionOutput.out());
printClientContextID(sessionOutput.out(), param);
- printSignature(sessionOutput.out());
+ printSignature(sessionOutput.out(), param);
printType(sessionOutput.out(), sessionConfig);
long errorCount = 1; // so far we just return 1 error
try {
@@ -498,7 +543,7 @@
} catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
handleExecuteStatementException(e, execution, param);
response.setStatus(execution.getHttpStatus());
- ResultUtil.printError(sessionOutput.out(), e);
+ printError(sessionOutput.out(), e);
ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
} finally {
// make sure that we stop buffering and return the result to the http response
@@ -533,6 +578,7 @@
getHyracksDataset(), resultProperties, stats, null, param.clientContextID, optionalParameters);
translator.compileAndExecute(getHyracksClientConnection(), queryCtx, requestParameters);
execution.end();
+ printExecutionPlans(sessionOutput, translator.getExecutionPlans());
}
protected void handleExecuteStatementException(Throwable t, RequestExecutionState state, RequestParameters param) {
@@ -566,4 +612,30 @@
state.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
}
}
+
+ protected void printError(PrintWriter sessionOut, Throwable throwable) {
+ ResultUtil.printError(sessionOut, throwable);
+ }
+
+ protected void printExecutionPlans(SessionOutput output, ExecutionPlans executionPlans) {
+ final PrintWriter pw = output.out();
+ pw.print("\t\"");
+ pw.print(ResultFields.PLANS.str());
+ pw.print("\":");
+ final SessionConfig.PlanFormat planFormat = output.config().getPlanFormat();
+ switch (planFormat) {
+ case JSON:
+ case STRING:
+ pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
+ break;
+ default:
+ throw new IllegalStateException("Unrecognized plan format: " + planFormat);
+ }
+ pw.print(",\n");
+ }
+
+ private static boolean isJsonFormat(String format) {
+ return format.startsWith(HttpUtil.ContentType.APPLICATION_JSON)
+ || format.equalsIgnoreCase(HttpUtil.ContentType.JSON);
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
index 8dd4290..6b632a1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/StorageApiServlet.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.api.http.server;
+import static org.apache.hyracks.util.NetworkUtil.toHostPort;
+
import java.io.IOException;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
@@ -131,8 +133,7 @@
ArrayNode replicasArray = OBJECT_MAPPER.createArrayNode();
for (IPartitionReplica replica : replicas) {
final ObjectNode replicaJson = OBJECT_MAPPER.createObjectNode();
- final InetSocketAddress location = replica.getIdentifier().getLocation();
- replicaJson.put("location", location.getHostString() + ":" + location.getPort());
+ replicaJson.put("location", toHostPort(replica.getIdentifier().getLocation()));
replicaJson.put("status", replica.getStatus().toString());
replicasArray.add(replicaJson);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
deleted file mode 100644
index 3650189..0000000
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/UpdateApiServlet.java
+++ /dev/null
@@ -1,53 +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.api.http.server;
-
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.translator.IStatementExecutorFactory;
-import org.apache.hyracks.http.api.IServletRequest;
-
-public class UpdateApiServlet extends RestApiServlet {
- private static final byte ALLOWED_CATEGORIES = Statement.Category.QUERY | Statement.Category.UPDATE;
-
- public UpdateApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, ICcApplicationContext appCtx,
- ILangCompilationProvider compilationProvider, IStatementExecutorFactory statementExecutorFactory,
- IStorageComponentProvider componentProvider) {
- super(ctx, paths, appCtx, compilationProvider, statementExecutorFactory, componentProvider);
- }
-
- @Override
- protected String getQueryParameter(IServletRequest request) {
- return request.getParameter("statements");
- }
-
- @Override
- protected byte getAllowedCategories() {
- return ALLOWED_CATEGORIES;
- }
-
- @Override
- protected String getErrorMessage() {
- return "Invalid statement: Non-Update statement %s to the Update API.";
- }
-}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
index e8bcb10..053e6cd 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveNotificationHandler.java
@@ -25,7 +25,6 @@
import org.apache.asterix.active.ActiveEvent;
import org.apache.asterix.active.ActiveEvent.Kind;
-import org.apache.asterix.active.ActivityState;
import org.apache.asterix.active.EntityId;
import org.apache.asterix.active.IActiveEntityEventsListener;
import org.apache.asterix.active.IActiveNotificationHandler;
@@ -33,7 +32,6 @@
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
-import org.apache.asterix.metadata.api.IActiveEntityController;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.utils.DatasetUtil;
@@ -214,14 +212,7 @@
for (IActiveEntityEventsListener listener : entityEventListeners.values()) {
synchronized (listener) {
LOGGER.log(level, "Entity " + listener.getEntityId() + " is " + listener.getStats());
- if (listener.getState() == ActivityState.PERMANENTLY_FAILED
- && listener instanceof IActiveEntityController) {
- LOGGER.log(level, "Recovering");
- ((IActiveEntityController) listener).recover();
- } else {
- LOGGER.log(level, "Only notifying");
- listener.notifyAll();
- }
+ listener.notifyAll();
}
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
index 1ba418a..7f632f0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/CCExtensionManager.java
@@ -28,12 +28,15 @@
import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
import org.apache.asterix.common.api.ExtensionId;
import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.compiler.provider.AqlCompilationProvider;
import org.apache.asterix.compiler.provider.ILangCompilationProvider;
import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
+import org.apache.asterix.hyracks.bootstrap.GlobalRecoveryManager;
import org.apache.asterix.om.functions.IFunctionExtensionManager;
import org.apache.asterix.om.functions.IFunctionManager;
import org.apache.asterix.runtime.functions.FunctionCollection;
@@ -41,6 +44,8 @@
import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.asterix.utils.ExtensionUtil;
import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.exceptions.HyracksDataException;
/**
@@ -53,6 +58,7 @@
private final ILangCompilationProvider aqlCompilationProvider;
private final ILangCompilationProvider sqlppCompilationProvider;
private final IFunctionManager functionManager;
+ private final IGlobalRecoveryExtension globalRecoveryExtension;
private transient IStatementExecutorFactory statementExecutorFactory;
/**
@@ -71,6 +77,7 @@
Pair<ExtensionId, ILangCompilationProvider> sqlppcp = null;
Pair<ExtensionId, IFunctionManager> fm = null;
IStatementExecutorExtension see = null;
+ IGlobalRecoveryExtension gre = null;
if (list != null) {
Set<ExtensionId> extensionIds = new HashSet<>();
for (AsterixExtension extensionConf : list) {
@@ -89,6 +96,9 @@
sqlppcp = ExtensionUtil.extendLangCompilationProvider(Language.SQLPP, sqlppcp, le);
fm = ExtensionUtil.extendFunctionManager(fm, le);
break;
+ case RECOVERY:
+ gre = (IGlobalRecoveryExtension) extension;
+ break;
default:
break;
}
@@ -99,6 +109,7 @@
this.sqlppCompilationProvider = sqlppcp == null ? new SqlppCompilationProvider() : sqlppcp.second;
this.functionManager =
fm == null ? new FunctionManager(FunctionCollection.createDefaultFunctionCollection()) : fm.second;
+ this.globalRecoveryExtension = gre;
}
/** @deprecated use getStatementExecutorFactory instead */
@@ -127,6 +138,14 @@
}
}
+ public IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+ IStorageComponentProvider componentProvider) {
+ if (globalRecoveryExtension == null) {
+ return new GlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+ }
+ return globalRecoveryExtension.getGlobalRecoveryManager(serviceCtx, hcc, componentProvider);
+ }
+
@Override
public IFunctionManager getFunctionManager() {
return functionManager;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.java
new file mode 100644
index 0000000..e9652a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/cc/IGlobalRecoveryExtension.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.app.cc;
+
+import org.apache.asterix.common.api.IExtension;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
+import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.hyracks.api.application.ICCServiceContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+
+/**
+ * An interface for extensions of {@code IGlobalRecoveryManager}
+ */
+public interface IGlobalRecoveryExtension extends IExtension {
+
+ @Override
+ default ExtensionKind getExtensionKind() {
+ return ExtensionKind.RECOVERY;
+ }
+
+ IGlobalRecoveryManager getGlobalRecoveryManager(ICCServiceContext serviceCtx, IHyracksClientConnection hcc,
+ IStorageComponentProvider componentProvider);
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 53d4f3f..977bbe3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -29,6 +29,7 @@
import org.apache.asterix.api.http.server.ResultUtil;
import org.apache.asterix.app.cc.CCExtensionManager;
import org.apache.asterix.app.translator.RequestParameters;
+import org.apache.asterix.common.api.Duration;
import org.apache.asterix.common.api.IClusterManagementWork;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.GlobalConfig;
@@ -64,7 +65,7 @@
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = LogManager.getLogger();
//TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2062
- public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
+ public static final long DEFAULT_NC_TIMEOUT_MILLIS = TimeUnit.MILLISECONDS.toMillis(Long.MAX_VALUE);
//TODO: Make configurable: https://issues.apache.org/jira/browse/ASTERIXDB-2063
public static final long DEFAULT_QUERY_CANCELLATION_WAIT_MILLIS = TimeUnit.MINUTES.toMillis(1);
private final String requestNodeId;
@@ -131,6 +132,7 @@
responseMsg.setResult(outWriter.toString());
responseMsg.setMetadata(outMetadata);
responseMsg.setStats(stats);
+ responseMsg.setExecutionPlans(translator.getExecutionPlans());
} catch (AlgebricksException | HyracksException | TokenMgrError
| org.apache.asterix.aqlplus.parser.TokenMgrError pe) {
// we trust that "our" exceptions are serializable and have a comprehensible error message
@@ -138,7 +140,7 @@
responseMsg.setError(pe);
} catch (Exception e) {
GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "Unexpected exception", e);
- responseMsg.setError(new Exception(e.toString()));
+ responseMsg.setError(e);
}
try {
messageBroker.sendApplicationMessageToNC(responseMsg, requestNodeId);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
index 7475be4..94dd541 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
@@ -23,6 +23,7 @@
import org.apache.asterix.common.messaging.api.INcAddressedMessage;
import org.apache.asterix.common.messaging.api.MessageFuture;
import org.apache.asterix.messaging.NCMessageBroker;
+import org.apache.asterix.translator.ExecutionPlans;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -39,6 +40,8 @@
private Throwable error;
+ private ExecutionPlans executionPlans;
+
public ExecuteStatementResponseMessage(long requestMessageId) {
this.requestMessageId = requestMessageId;
}
@@ -84,6 +87,14 @@
this.stats = stats;
}
+ public ExecutionPlans getExecutionPlans() {
+ return executionPlans;
+ }
+
+ public void setExecutionPlans(ExecutionPlans executionPlans) {
+ this.executionPlans = executionPlans;
+ }
+
@Override
public String toString() {
return String.format("%s(id=%s): %d characters", getClass().getSimpleName(), requestMessageId,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
new file mode 100644
index 0000000..db34131
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/HaltCallback.java
@@ -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.
+ */
+package org.apache.asterix.app.nc;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.util.ExitUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class HaltCallback implements IIoOperationFailedCallback {
+ private static final Logger LOGGER = LogManager.getLogger();
+ public static final HaltCallback INSTANCE = new HaltCallback();
+
+ private HaltCallback() {
+ }
+
+ @Override
+ public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+ LOGGER.error("IO Scheduler has failed", failure);
+ ExitUtil.halt(ExitUtil.EC_IO_SCHEDULER_FAILED);
+ }
+
+ @Override
+ public void operationFailed(ILSMIOOperation operation, Throwable t) {
+ LOGGER.error("Operation {} has failed", t);
+ if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ ExitUtil.halt(ExitUtil.EC_FLUSH_FAILED);
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
index a012f1e..c1a81a3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManager.java
@@ -22,6 +22,7 @@
import java.io.File;
import java.io.FilenameFilter;
import java.io.IOException;
+import java.nio.channels.ClosedByInterruptException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -54,29 +55,36 @@
}
@Override
- public synchronized void init(long lsn) throws HyracksDataException {
- final List<IndexCheckpoint> checkpoints = getCheckpoints();
+ public synchronized void init(String lastComponentTimestamp, long lsn) throws HyracksDataException {
+ List<IndexCheckpoint> checkpoints;
+ try {
+ checkpoints = getCheckpoints();
+ } catch (ClosedByInterruptException e) {
+ throw HyracksDataException.create(e);
+ }
if (!checkpoints.isEmpty()) {
LOGGER.warn(() -> "Checkpoints found on initializing: " + indexPath);
delete();
}
- IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(lsn);
+ IndexCheckpoint firstCheckpoint = IndexCheckpoint.first(lastComponentTimestamp, lsn);
persist(firstCheckpoint);
}
@Override
- public synchronized void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException {
+ public synchronized void replicated(String componentTimestamp, long masterLsn, long componentId)
+ throws HyracksDataException {
final Long localLsn = getLatest().getMasterNodeFlushMap().get(masterLsn);
if (localLsn == null) {
throw new IllegalStateException("Component flushed before lsn mapping was received");
}
- flushed(componentTimestamp, localLsn);
+ flushed(componentTimestamp, localLsn, componentId);
}
@Override
- public synchronized void flushed(String componentTimestamp, long lsn) throws HyracksDataException {
+ public synchronized void flushed(String componentTimestamp, long lsn, long componentId)
+ throws HyracksDataException {
final IndexCheckpoint latest = getLatest();
- IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentTimestamp);
+ IndexCheckpoint nextCheckpoint = IndexCheckpoint.next(latest, lsn, componentTimestamp, componentId);
persist(nextCheckpoint);
deleteHistory(nextCheckpoint.getId(), HISTORY_CHECKPOINTS);
}
@@ -85,19 +93,19 @@
public synchronized void masterFlush(long masterLsn, long localLsn) throws HyracksDataException {
final IndexCheckpoint latest = getLatest();
latest.getMasterNodeFlushMap().put(masterLsn, localLsn);
- final IndexCheckpoint next =
- IndexCheckpoint.next(latest, latest.getLowWatermark(), latest.getValidComponentTimestamp());
+ final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+ latest.getValidComponentTimestamp(), latest.getLastComponentId());
persist(next);
notifyAll();
}
@Override
- public synchronized long getLowWatermark() {
+ public synchronized long getLowWatermark() throws HyracksDataException {
return getLatest().getLowWatermark();
}
@Override
- public synchronized boolean isFlushed(long masterLsn) {
+ public synchronized boolean isFlushed(long masterLsn) throws HyracksDataException {
if (masterLsn == BULKLOAD_LSN) {
return true;
}
@@ -110,18 +118,28 @@
}
@Override
- public Optional<String> getValidComponentTimestamp() {
- final String validComponentTimestamp = getLatest().getValidComponentTimestamp();
+ public Optional<String> getValidComponentTimestamp() throws HyracksDataException {
+ String validComponentTimestamp = getLatest().getValidComponentTimestamp();
return validComponentTimestamp != null ? Optional.of(validComponentTimestamp) : Optional.empty();
}
@Override
- public int getCheckpointCount() {
- return getCheckpoints().size();
+ public int getCheckpointCount() throws HyracksDataException {
+ try {
+ return getCheckpoints().size();
+ } catch (ClosedByInterruptException e) {
+ throw HyracksDataException.create(e);
+ }
}
- private IndexCheckpoint getLatest() {
- final List<IndexCheckpoint> checkpoints = getCheckpoints();
+ @Override
+ public synchronized IndexCheckpoint getLatest() throws HyracksDataException {
+ List<IndexCheckpoint> checkpoints;
+ try {
+ checkpoints = getCheckpoints();
+ } catch (ClosedByInterruptException e) {
+ throw HyracksDataException.create(e);
+ }
if (checkpoints.isEmpty()) {
throw new IllegalStateException("Couldn't find any checkpoints for resource: " + indexPath);
}
@@ -129,13 +147,34 @@
return checkpoints.get(0);
}
- private List<IndexCheckpoint> getCheckpoints() {
+ @Override
+ public synchronized void setLastComponentId(long componentId) throws HyracksDataException {
+ final IndexCheckpoint latest = getLatest();
+ final IndexCheckpoint next = IndexCheckpoint.next(latest, latest.getLowWatermark(),
+ latest.getValidComponentTimestamp(), componentId);
+ persist(next);
+ }
+
+ @Override
+ public synchronized void advanceValidComponentTimestamp(String timestamp) throws HyracksDataException {
+ final IndexCheckpoint latest = getLatest();
+ if (latest.getValidComponentTimestamp() == null
+ || timestamp.compareTo(latest.getValidComponentTimestamp()) > 0) {
+ final IndexCheckpoint next =
+ IndexCheckpoint.next(latest, latest.getLowWatermark(), timestamp, latest.getLastComponentId());
+ persist(next);
+ }
+ }
+
+ private List<IndexCheckpoint> getCheckpoints() throws ClosedByInterruptException {
List<IndexCheckpoint> checkpoints = new ArrayList<>();
final File[] checkpointFiles = indexPath.toFile().listFiles(CHECKPOINT_FILE_FILTER);
if (checkpointFiles != null) {
for (File checkpointFile : checkpointFiles) {
try {
checkpoints.add(read(checkpointFile.toPath()));
+ } catch (ClosedByInterruptException e) {
+ throw e;
} catch (IOException e) {
LOGGER.warn(() -> "Couldn't read index checkpoint file: " + checkpointFile, e);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
index 19ad8f6..e0b3105 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/IndexCheckpointManagerProvider.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.storage.IIndexCheckpointManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.IIOManager;
@@ -53,14 +54,10 @@
private IndexCheckpointManager create(ResourceReference ref) {
try {
- final Path indexPath = getIndexPath(ref);
+ final Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
return new IndexCheckpointManager(indexPath);
} catch (HyracksDataException e) {
throw new IllegalStateException(e);
}
}
-
- private Path getIndexPath(ResourceReference indexRef) throws HyracksDataException {
- return ioManager.resolve(indexRef.getRelativePath().toString()).getFile().toPath();
- }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 288e5f2..c3201c3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -52,7 +52,6 @@
import org.apache.asterix.common.context.DatasetLifecycleManager;
import org.apache.asterix.common.context.DatasetMemoryManager;
import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.replication.IReplicationChannel;
@@ -61,6 +60,7 @@
import org.apache.asterix.common.storage.IReplicaManager;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.external.library.ExternalLibraryManager;
import org.apache.asterix.file.StorageComponentProvider;
@@ -169,7 +169,7 @@
}
@Override
- public void initialize(boolean initialRun) throws IOException, ACIDException {
+ public void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun) throws IOException {
ioManager = getServiceContext().getIoManager();
threadExecutor =
MaintainedThreadNameExecutorService.newCachedThreadPool(getServiceContext().getThreadFactory());
@@ -177,20 +177,14 @@
IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
IPageReplacementStrategy prs = new ClockPageReplacementStrategy(allocator,
storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages());
-
- AsynchronousScheduler.INSTANCE.init(getServiceContext().getThreadFactory());
- lsmIOScheduler = AsynchronousScheduler.INSTANCE;
-
+ lsmIOScheduler = new AsynchronousScheduler(getServiceContext().getThreadFactory(), HaltCallback.INSTANCE);
metadataMergePolicyFactory = new PrefixMergePolicyFactory();
indexCheckpointManagerProvider = new IndexCheckpointManagerProvider(ioManager);
-
ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory =
new PersistentLocalResourceRepositoryFactory(ioManager, indexCheckpointManagerProvider);
-
localResourceRepository =
(PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory.createRepository();
-
- txnSubsystem = new TransactionSubsystem(this);
+ txnSubsystem = new TransactionSubsystem(this, recoveryManagerFactory);
IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
SystemState systemState = recoveryMgr.getSystemState();
if (initialRun || systemState == SystemState.PERMANENT_DATA_LOSS) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index d4e652d..5e8a5e8 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -48,7 +48,7 @@
import org.apache.asterix.common.context.IndexInfo;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
@@ -77,6 +77,8 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -102,14 +104,14 @@
private static final String RECOVERY_FILES_DIR_NAME = "recovery_temp";
private Map<Long, JobEntityCommits> jobId2WinnerEntitiesMap = null;
private final long cachedEntityCommitsPerJobSize;
- private final PersistentLocalResourceRepository localResourceRepository;
+ protected final PersistentLocalResourceRepository localResourceRepository;
private final ICheckpointManager checkpointManager;
private SystemState state;
- private final INCServiceContext serviceCtx;
- private final INcApplicationContext appCtx;
+ protected final INCServiceContext serviceCtx;
+ protected final INcApplicationContext appCtx;
private static final TxnId recoveryTxnId = new TxnId(-1);
- public RecoveryManager(ITransactionSubsystem txnSubsystem, INCServiceContext serviceCtx) {
+ public RecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem) {
this.serviceCtx = serviceCtx;
this.txnSubsystem = txnSubsystem;
this.appCtx = txnSubsystem.getApplicationContext();
@@ -225,6 +227,7 @@
break;
case LogType.FLUSH:
case LogType.WAIT:
+ case LogType.WAIT_FOR_FLUSHES:
case LogType.MARKER:
case LogType.FILTER:
break;
@@ -392,7 +395,6 @@
logRecord = logReader.next();
continue;
}
- idGenerator.refresh();
DatasetInfo dsInfo = datasetLifecycleManager.getDatasetInfo(datasetId);
// we only need to flush open indexes here (opened by previous update records)
// if an index has no ongoing updates, then it's memory component must be empty
@@ -401,24 +403,15 @@
if (iInfo.isOpen() && iInfo.getPartition() == partition) {
maxDiskLastLsn = resourceId2MaxLSNMap.get(iInfo.getResourceId());
index = iInfo.getIndex();
- AbstractLSMIOOperationCallback ioCallback =
- (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
if (logRecord.getLSN() > maxDiskLastLsn
&& !index.isCurrentMutableComponentEmpty()) {
// schedule flush
- ioCallback.updateLastLSN(logRecord.getLSN());
redoFlush(index, logRecord);
redoCount++;
} else {
- if (index.isMemoryComponentsAllocated()) {
- // if the memory component has been allocated, we
- // force it to receive the same Id
- index.getCurrentMemoryComponent().resetId(idGenerator.getId(), true);
- } else {
- // otherwise, we refresh the id stored in ioCallback
- // to ensure the memory component receives correct Id upon activation
- ioCallback.forceRefreshNextId();
- }
+ // otherwise, do nothing since this component had no records when flush was
+ // scheduled.. TODO: update checkpoint file? and do the
+ // lsn checks from the checkpoint file
}
}
}
@@ -428,6 +421,7 @@
case LogType.ENTITY_COMMIT:
case LogType.ABORT:
case LogType.WAIT:
+ case LogType.WAIT_FOR_FLUSHES:
case LogType.MARKER:
//do nothing
break;
@@ -473,10 +467,10 @@
long minFirstLSN = logMgr.getAppendLSN();
if (!openIndexList.isEmpty()) {
for (IIndex index : openIndexList) {
- AbstractLSMIOOperationCallback ioCallback =
- (AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
+ LSMIOOperationCallback ioCallback =
+ (LSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback();
if (!((AbstractLSMIndex) index).isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()) {
- firstLSN = ioCallback.getFirstLSN();
+ firstLSN = ioCallback.getPersistenceLsn();
minFirstLSN = Math.min(minFirstLSN, firstLSN);
}
}
@@ -684,6 +678,7 @@
case LogType.FLUSH:
case LogType.FILTER:
case LogType.WAIT:
+ case LogType.WAIT_FOR_FLUSHES:
case LogType.MARKER:
//ignore
break;
@@ -824,10 +819,15 @@
}
private static void redoFlush(ILSMIndex index, ILogRecord logRecord) throws HyracksDataException {
+ long flushLsn = logRecord.getLSN();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ accessor.getOpContext().setParameters(flushMap);
long minId = logRecord.getFlushingComponentMinId();
long maxId = logRecord.getFlushingComponentMaxId();
ILSMComponentId id = new LSMComponentId(minId, maxId);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, index.getCurrentMemoryComponent().getId());
if (!index.getDiskComponents().isEmpty()) {
ILSMDiskComponent diskComponent = index.getDiskComponents().get(0);
ILSMComponentId maxDiskComponentId = diskComponent.getId();
@@ -837,7 +837,17 @@
}
}
index.getCurrentMemoryComponent().resetId(id, true);
- accessor.scheduleFlush(index.getIOOperationCallback());
+ ILSMIOOperation flush = accessor.scheduleFlush();
+ try {
+ flush.sync();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(flush.getFailure());
+ }
+ index.resetCurrentComponentIndex();
}
private class JobEntityCommits {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
index 8158096..79c87c0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/TransactionSubsystem.java
@@ -30,6 +30,7 @@
import org.apache.asterix.common.transactions.ILockManager;
import org.apache.asterix.common.transactions.ILogManager;
import org.apache.asterix.common.transactions.IRecoveryManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
import org.apache.asterix.common.transactions.ITransactionManager;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.asterix.transaction.management.service.locking.ConcurrentLockManager;
@@ -59,7 +60,7 @@
private long profilerEntityCommitLogCount = 0;
private EntityCommitProfiler ecp;
- public TransactionSubsystem(INcApplicationContext appCtx) {
+ public TransactionSubsystem(INcApplicationContext appCtx, IRecoveryManagerFactory recoveryManagerFactory) {
this.appCtx = appCtx;
this.id = appCtx.getServiceContext().getNodeId();
this.txnProperties = appCtx.getTransactionProperties();
@@ -78,7 +79,7 @@
}
this.logManager = replicationEnabled ? new LogManagerWithReplication(this) : new LogManager(this);
- this.recoveryManager = new RecoveryManager(this, appCtx.getServiceContext());
+ this.recoveryManager = recoveryManagerFactory.createRecoveryManager(appCtx.getServiceContext(), this);
if (txnProperties.isCommitProfilerEnabled()) {
ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
((ExecutorService) appCtx.getThreadExecutor()).submit(ecp);
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 e983763..524c684 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
@@ -156,6 +156,8 @@
import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
import org.apache.asterix.translator.CompiledStatements.CompiledUpsertStatement;
import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansHtmlPrintUtil;
import org.apache.asterix.translator.IRequestParameters;
import org.apache.asterix.translator.IStatementExecutor;
import org.apache.asterix.translator.IStatementExecutorContext;
@@ -1775,6 +1777,7 @@
new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls);
+ afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
if (spec != null) {
@@ -1866,6 +1869,7 @@
stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
stmtDelete.getQuery());
JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs);
+ afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
@@ -2372,6 +2376,7 @@
metadataProvider.setMetadataTxnContext(mdTxnCtx);
try {
final JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null);
+ afterCompile();
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
bActiveTxn = false;
return query.isExplain() || !sessionConfig.isExecuteQuery() ? null : jobSpec;
@@ -2780,6 +2785,11 @@
return (dataverse != null) ? dataverse : activeDataverse.getDataverseName();
}
+ @Override
+ public ExecutionPlans getExecutionPlans() {
+ return apiFramework.getExecutionPlans();
+ }
+
public String getActiveDataverse(Identifier dataverse) {
return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
}
@@ -2820,4 +2830,10 @@
throw new MetadataException(ErrorCode.CANNOT_DROP_INDEX, index.getIndexName(), index.getDatasetName());
}
}
+
+ protected void afterCompile() {
+ if (sessionOutput.config().is(SessionConfig.FORMAT_HTML)) {
+ ExecutionPlansHtmlPrintUtil.print(sessionOutput.out(), getExecutionPlans());
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 699892e..8f78ce1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -37,11 +37,8 @@
import org.apache.asterix.api.http.server.ClusterApiServlet;
import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
import org.apache.asterix.api.http.server.ConnectorApiServlet;
-import org.apache.asterix.api.http.server.DdlApiServlet;
import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
-import org.apache.asterix.api.http.server.FullApiServlet;
import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
-import org.apache.asterix.api.http.server.QueryApiServlet;
import org.apache.asterix.api.http.server.QueryCancellationServlet;
import org.apache.asterix.api.http.server.QueryResultApiServlet;
import org.apache.asterix.api.http.server.QueryServiceServlet;
@@ -50,7 +47,6 @@
import org.apache.asterix.api.http.server.RebalanceApiServlet;
import org.apache.asterix.api.http.server.ServletConstants;
import org.apache.asterix.api.http.server.ShutdownApiServlet;
-import org.apache.asterix.api.http.server.UpdateApiServlet;
import org.apache.asterix.api.http.server.VersionApiServlet;
import org.apache.asterix.app.active.ActiveNotificationHandler;
import org.apache.asterix.app.cc.CCExtensionManager;
@@ -58,7 +54,9 @@
import org.apache.asterix.app.replication.NcLifecycleCoordinator;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
import org.apache.asterix.common.config.AsterixExtension;
+import org.apache.asterix.common.config.ExtensionProperties;
import org.apache.asterix.common.config.ExternalProperties;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.config.MetadataProperties;
@@ -66,6 +64,7 @@
import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.replication.INcLifecycleCoordinator;
import org.apache.asterix.common.utils.Servlets;
@@ -143,12 +142,13 @@
INcLifecycleCoordinator lifecycleCoordinator = createNcLifeCycleCoordinator(repProp.isReplicationEnabled());
ExternalLibraryUtils.setUpExternaLibraries(libraryManager, false);
componentProvider = new StorageComponentProvider();
- GlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
+
+ List<AsterixExtension> extensions = new ArrayList<>();
+ extensions.addAll(getExtensions());
+ ccExtensionManager = new CCExtensionManager(extensions);
+ IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
statementExecutorCtx = new StatementExecutorContext();
appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator);
- List<AsterixExtension> extensions = new ArrayList<>();
- extensions.addAll(this.getExtensions());
- ccExtensionManager = new CCExtensionManager(extensions);
appCtx.setExtensionManager(ccExtensionManager);
final CCConfig ccConfig = controllerService.getCCConfig();
if (System.getProperty("java.rmi.server.hostname") == null) {
@@ -174,15 +174,15 @@
}
protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
- GlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
+ IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator)
throws AlgebricksException, IOException {
return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
new MetadataLockManager());
}
- protected GlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
- return new GlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
+ protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
+ return ccExtensionManager.getGlobalRecoveryManager(ccServiceCtx, getHcc(), componentProvider);
}
protected INcLifecycleCoordinator createNcLifeCycleCoordinator(boolean replicationEnabled) {
@@ -195,8 +195,8 @@
LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
}
- protected List<AsterixExtension> getExtensions() {
- return appCtx.getExtensionProperties().getExtensions();
+ protected List<AsterixExtension> getExtensions() throws Exception {
+ return new ExtensionProperties(PropertiesAccessor.getInstance(ccServiceCtx.getAppConfig())).getExtensions();
}
protected void configureServers() throws Exception {
@@ -234,18 +234,6 @@
jsonAPIServer.setAttribute(ServletConstants.RUNNING_QUERIES_ATTR, statementExecutorCtx);
jsonAPIServer.setAttribute(ServletConstants.SERVICE_CONTEXT_ATTR, ccServiceCtx);
- // AQL rest APIs.
- addServlet(jsonAPIServer, Servlets.AQL_QUERY);
- addServlet(jsonAPIServer, Servlets.AQL_UPDATE);
- addServlet(jsonAPIServer, Servlets.AQL_DDL);
- addServlet(jsonAPIServer, Servlets.AQL);
-
- // SQL+x+ rest APIs.
- addServlet(jsonAPIServer, Servlets.SQLPP_QUERY);
- addServlet(jsonAPIServer, Servlets.SQLPP_UPDATE);
- addServlet(jsonAPIServer, Servlets.SQLPP_DDL);
- addServlet(jsonAPIServer, Servlets.SQLPP);
-
// Other APIs.
addServlet(jsonAPIServer, Servlets.QUERY_STATUS);
addServlet(jsonAPIServer, Servlets.QUERY_RESULT);
@@ -278,30 +266,6 @@
protected IServlet createServlet(ConcurrentMap<String, Object> ctx, String key, String... paths) {
switch (key) {
- case Servlets.AQL:
- return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.AQL_QUERY:
- return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.AQL_UPDATE:
- return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.AQL_DDL:
- return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(AQL),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.SQLPP:
- return new FullApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.SQLPP_QUERY:
- return new QueryApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.SQLPP_UPDATE:
- return new UpdateApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
- getStatementExecutorFactory(), componentProvider);
- case Servlets.SQLPP_DDL:
- return new DdlApiServlet(ctx, paths, appCtx, ccExtensionManager.getCompilationProvider(SQLPP),
- getStatementExecutorFactory(), componentProvider);
case Servlets.RUNNING_REQUESTS:
return new QueryCancellationServlet(ctx, paths);
case Servlets.QUERY_STATUS:
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
index 3d9b822..5fc5c57 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/GlobalRecoveryManager.java
@@ -56,7 +56,7 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final IStorageComponentProvider componentProvider;
protected final ICCServiceContext serviceCtx;
- protected IHyracksClientConnection hcc;
+ protected final IHyracksClientConnection hcc;
protected volatile boolean recoveryCompleted;
protected volatile boolean recovering;
@@ -126,7 +126,9 @@
throws Exception {
// Loop over datasets
for (Dataverse dataverse : MetadataManager.INSTANCE.getDataverses(mdTxnCtx)) {
- mdTxnCtx = recoverDataset(appCtx, mdTxnCtx, dataverse);
+ mdTxnCtx = recoverDatasets(appCtx, mdTxnCtx, dataverse);
+ // Fixes ASTERIXDB-2386 by caching the dataverse during recovery
+ MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse.getDataverseName());
}
return mdTxnCtx;
}
@@ -138,8 +140,8 @@
}
}
- private MetadataTransactionContext recoverDataset(ICcApplicationContext appCtx, MetadataTransactionContext mdTxnCtx,
- Dataverse dataverse) throws Exception {
+ private MetadataTransactionContext recoverDatasets(ICcApplicationContext appCtx,
+ MetadataTransactionContext mdTxnCtx, Dataverse dataverse) throws Exception {
if (!dataverse.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
MetadataProvider metadataProvider = new MetadataProvider(appCtx, dataverse);
try {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 494198b..b8c7e15 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -27,6 +27,7 @@
import org.apache.asterix.api.http.server.ServletConstants;
import org.apache.asterix.api.http.server.StorageApiServlet;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.nc.RecoveryManager;
import org.apache.asterix.app.replication.message.RegistrationTasksRequestMessage;
import org.apache.asterix.common.api.AsterixThreadFactory;
import org.apache.asterix.common.api.INcApplicationContext;
@@ -44,6 +45,7 @@
import org.apache.asterix.common.transactions.Checkpoint;
import org.apache.asterix.common.transactions.IRecoveryManager;
import org.apache.asterix.common.transactions.IRecoveryManager.SystemState;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
import org.apache.asterix.common.utils.PrintUtil;
import org.apache.asterix.common.utils.Servlets;
import org.apache.asterix.common.utils.StorageConstants;
@@ -121,7 +123,7 @@
}
updateOnNodeJoin();
}
- runtimeContext.initialize(runtimeContext.getNodeProperties().isInitialRun());
+ runtimeContext.initialize(getRecoveryManagerFactory(), runtimeContext.getNodeProperties().isInitialRun());
MessagingProperties messagingProperties = runtimeContext.getMessagingProperties();
IMessageBroker messageBroker = new NCMessageBroker(controllerService, messagingProperties);
this.ncServiceCtx.setMessageBroker(messageBroker);
@@ -144,6 +146,10 @@
performLocalCleanUp();
}
+ protected IRecoveryManagerFactory getRecoveryManagerFactory() {
+ return RecoveryManager::new;
+ }
+
@Override
protected void configureLoggingLevel(Level level) {
super.configureLoggingLevel(level);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index acc3970..4b245a8 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -30,6 +30,7 @@
import java.nio.file.attribute.BasicFileAttributes;
import java.util.ArrayList;
import java.util.List;
+import java.util.concurrent.TimeUnit;
import java.util.function.BiPredicate;
import java.util.stream.Stream;
@@ -70,6 +71,7 @@
joinPath(getProjectPath().toString(), "src", "test", "resources", "cc.conf");
private static final String DEFAULT_STORAGE_PATH = joinPath("target", "io", "dir");
private static String storagePath = DEFAULT_STORAGE_PATH;
+ private static final long RESULT_TTL = TimeUnit.MINUTES.toMillis(5);
static {
System.setProperty("java.util.logging.manager", org.apache.logging.log4j.jul.LogManager.class.getName());
@@ -95,7 +97,8 @@
* main method to run a simple 2 node cluster in-process
* suggested VM arguments: <code>-enableassertions -Xmx2048m -Dfile.encoding=UTF-8</code>
*
- * @param args unused
+ * @param args
+ * unused
*/
public static void main(String[] args) throws Exception {
TestUtils.redirectLoggingToConsole();
@@ -198,7 +201,7 @@
ccConfig.setClientListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
ccConfig.setClientListenPort(DEFAULT_HYRACKS_CC_CLIENT_PORT);
ccConfig.setClusterListenPort(DEFAULT_HYRACKS_CC_CLUSTER_PORT);
- ccConfig.setResultTTL(120000L);
+ ccConfig.setResultTTL(RESULT_TTL);
ccConfig.setResultSweepThreshold(1000L);
ccConfig.setEnforceFrameWriterProtocol(true);
configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb"));
@@ -217,18 +220,21 @@
ncConfig.setDataListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
ncConfig.setResultListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
ncConfig.setMessagingListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
- ncConfig.setResultTTL(120000L);
+ ncConfig.setResultTTL(RESULT_TTL);
ncConfig.setResultSweepThreshold(1000L);
ncConfig.setVirtualNC();
configManager.set(ControllerConfig.Option.DEFAULT_DIR, joinPath(getDefaultStoragePath(), "asterixdb", ncName));
return ncConfig;
}
- protected INCApplication createNCApplication() {
+ protected INCApplication createNCApplication()
+ throws InstantiationException, IllegalAccessException, ClassNotFoundException {
+ // Instead of using this flag, RecoveryManagerTest should set the desired class in its config file
if (!gracefulShutdown) {
return new UngracefulShutdownNCApplication();
}
- return new NCApplication();
+ String ncAppClass = (String) configManager.get(NCConfig.Option.APP_CLASS);
+ return (INCApplication) Class.forName(ncAppClass).newInstance();
}
private NCConfig fixupIODevices(NCConfig ncConfig) throws IOException, AsterixException, CmdLineException {
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 5cda9f2..5be349e 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
@@ -37,7 +37,6 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
import org.apache.asterix.common.transactions.ITransactionManager;
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -58,17 +57,18 @@
import org.apache.asterix.runtime.formats.FormatUtils;
import org.apache.asterix.runtime.formats.NonTaggedDataFormat;
import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
import org.apache.asterix.runtime.operators.LSMPrimaryUpsertOperatorNodePushable;
import org.apache.asterix.runtime.utils.CcApplicationContext;
import org.apache.asterix.test.runtime.ExecutionTestUtil;
-import org.apache.asterix.transaction.management.opcallbacks.AbstractIndexModificationOperationCallback.Operation;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
import org.apache.asterix.transaction.management.runtime.CommitRuntime;
import org.apache.asterix.transaction.management.service.logging.LogReader;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.operators.base.SinkRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
import org.apache.hyracks.algebricks.runtime.operators.std.EmptyTupleSourceRuntimeFactory;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -166,7 +166,7 @@
public void deInit(boolean cleanupOnStop) throws Exception {
ExternalUDFLibrarian.removeLibraryDir();
- ExecutionTestUtil.tearDown(cleanupOnStop);
+ ExecutionTestUtil.tearDown(cleanupOnStop, runHDFS);
}
public void setOpts(List<Pair<IOption, Object>> opts) {
@@ -186,7 +186,7 @@
return new TxnId(jobId.getId());
}
- public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+ public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
StorageComponentProvider storageComponentProvider, Index secondaryIndex)
@@ -195,7 +195,39 @@
primaryKeyIndicators, storageComponentProvider, secondaryIndex, IndexOperation.INSERT);
}
- public Pair<LSMInsertDeleteOperatorNodePushable, CommitRuntime> getInsertPipeline(IHyracksTaskContext ctx,
+ public Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> getBulkLoadSecondaryOperator(
+ IHyracksTaskContext ctx, Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType,
+ ARecordType metaType, int[] filterFields, int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
+ StorageComponentProvider storageComponentProvider, Index secondaryIndex, int numElementsHint)
+ throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+ try {
+ MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+ org.apache.hyracks.algebricks.common.utils.Pair<ILSMMergePolicyFactory, Map<String, String>> mergePolicy =
+ DatasetUtil.getMergePolicyFactory(dataset, mdTxnCtx);
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
+ mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
+ SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
+ IIndexDataflowHelperFactory secondaryIndexHelperFactory = new IndexDataflowHelperFactory(
+ storageComponentProvider.getStorageManager(), secondaryIndexInfo.fileSplitProvider);
+ IIndexDataflowHelperFactory primaryIndexHelperFactory = new IndexDataflowHelperFactory(
+ storageComponentProvider.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+ int[] fieldPermutation = new int[secondaryIndex.getKeyFieldNames().size()];
+ for (int i = 0; i < fieldPermutation.length; i++) {
+ fieldPermutation[i] = i;
+ }
+ LSMIndexBulkLoadOperatorNodePushable op =
+ new LSMIndexBulkLoadOperatorNodePushable(secondaryIndexHelperFactory, primaryIndexHelperFactory,
+ ctx, 0, fieldPermutation, 1.0F, false, numElementsHint, true, secondaryIndexInfo.rDesc,
+ BulkLoadUsage.CREATE_INDEX, dataset.getDatasetId());
+ op.setOutputFrameWriter(0, new SinkRuntimeFactory().createPushRuntime(ctx)[0], null);
+ return Pair.of(secondaryIndexInfo, op);
+ } catch (Throwable th) {
+ throw HyracksDataException.create(th);
+ }
+ }
+
+ public Pair<LSMInsertDeleteOperatorNodePushable, IPushRuntime> getInsertPipeline(IHyracksTaskContext ctx,
Dataset dataset, IAType[] primaryKeyTypes, ARecordType recordType, ARecordType metaType, int[] filterFields,
int[] primaryKeyIndexes, List<Integer> primaryKeyIndicators,
StorageComponentProvider storageComponentProvider, Index secondaryIndex, IndexOperation op)
@@ -210,10 +242,8 @@
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
PrimaryIndexInfo primaryIndexInfo = new PrimaryIndexInfo(dataset, primaryKeyTypes, recordType, metaType,
mergePolicy.first, mergePolicy.second, filterFields, primaryKeyIndexes, primaryKeyIndicators);
- IModificationOperationCallbackFactory modOpCallbackFactory =
- new PrimaryIndexModificationOperationCallbackFactory(dataset.getDatasetId(),
- primaryIndexInfo.primaryKeyIndexes, TXN_SUBSYSTEM_PROVIDER, Operation.get(op),
- ResourceType.LSM_BTREE);
+ IModificationOperationCallbackFactory modOpCallbackFactory = dataset.getModificationCallbackFactory(
+ storageComponentProvider, primaryIndexInfo.index, op, primaryIndexInfo.primaryKeyIndexes);
IRecordDescriptorProvider recordDescProvider = primaryIndexInfo.getInsertRecordDescriptorProvider();
RecordDescriptor recordDesc =
recordDescProvider.getInputRecordDescriptor(new ActivityId(new OperatorDescriptorId(0), 0), 0);
@@ -272,16 +302,18 @@
secondaryIndexInfo.insertFieldsPermutations, secondaryIndexInfo.rDesc, op, false,
secondaryIndexHelperFactory, secondaryModCallbackFactory, null);
assignOp.setOutputFrameWriter(0, secondaryInsertOp, secondaryIndexInfo.rDesc);
- CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
- secondaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
- true);
+
+ IPushRuntime commitOp =
+ dataset.getCommitRuntimeFactory(mdProvider, secondaryIndexInfo.primaryKeyIndexes, true)
+ .createPushRuntime(ctx)[0];
+
secondaryInsertOp.setOutputFrameWriter(0, commitOp, secondaryIndexInfo.rDesc);
commitOp.setInputRecordDescriptor(0, secondaryIndexInfo.rDesc);
return Pair.of(insertOp, commitOp);
} else {
- CommitRuntime commitOp = new CommitRuntime(ctx, getTxnJobId(ctx), dataset.getDatasetId(),
- primaryIndexInfo.primaryKeyIndexes, true, ctx.getTaskAttemptId().getTaskId().getPartition(),
- true);
+ IPushRuntime commitOp =
+ dataset.getCommitRuntimeFactory(mdProvider, primaryIndexInfo.primaryKeyIndexes, true)
+ .createPushRuntime(ctx)[0];
insertOp.setOutputFrameWriter(0, commitOp, primaryIndexInfo.rDesc);
commitOp.setInputRecordDescriptor(0, primaryIndexInfo.rDesc);
return Pair.of(insertOp, commitOp);
@@ -380,7 +412,6 @@
(ICcApplicationContext) ExecutionTestUtil.integrationUtil.cc.getApplicationContext(), dataverse);
SecondaryIndexInfo secondaryIndexInfo = new SecondaryIndexInfo(primaryIndexInfo, secondaryIndex);
try {
-
IResourceFactory resourceFactory = primaryIndexInfo.dataset.getResourceFactory(mdProvider, secondaryIndex,
primaryIndexInfo.recordType, primaryIndexInfo.metaType, mergePolicy.first, mergePolicy.second);
IndexBuilderFactory indexBuilderFactory =
@@ -419,9 +450,9 @@
secondaryIndexSerdes[i] =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(secondaryKeyTypes[i]);
}
- for (; i < primaryKeyTypes.length; i++) {
- secondaryIndexSerdes[i] =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(primaryKeyTypes[i]);
+ for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+ secondaryIndexSerdes[i] = SerializerDeserializerProvider.INSTANCE
+ .getSerializerDeserializer(primaryKeyTypes[i - secondaryKeyTypes.length]);
}
return secondaryIndexSerdes;
}
@@ -447,8 +478,9 @@
for (; i < secondaryKeyTypes.length; i++) {
secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(secondaryKeyTypes[i]);
}
- for (; i < primaryKeyTypes.length; i++) {
- secondaryIndexTypeTraits[i] = TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i]);
+ for (; i < secondaryKeyTypes.length + primaryKeyTypes.length; i++) {
+ secondaryIndexTypeTraits[i] =
+ TypeTraitProvider.INSTANCE.getTypeTrait(primaryKeyTypes[i - secondaryKeyTypes.length]);
}
return secondaryIndexTypeTraits;
}
@@ -491,12 +523,14 @@
}
public static class SecondaryIndexInfo {
- private final int[] primaryKeyIndexes;
- private final PrimaryIndexInfo primaryIndexInfo;
- private final Index secondaryIndex;
- private final ConstantFileSplitProvider fileSplitProvider;
- private final RecordDescriptor rDesc;
- private final int[] insertFieldsPermutations;
+ final int[] primaryKeyIndexes;
+ final PrimaryIndexInfo primaryIndexInfo;
+ final Index secondaryIndex;
+ final ConstantFileSplitProvider fileSplitProvider;
+ final ISerializerDeserializer<?>[] secondaryIndexSerdes;
+ final RecordDescriptor rDesc;
+ final int[] insertFieldsPermutations;
+ final ITypeTraits[] secondaryIndexTypeTraits;
public SecondaryIndexInfo(PrimaryIndexInfo primaryIndexInfo, Index secondaryIndex) {
this.primaryIndexInfo = primaryIndexInfo;
@@ -507,11 +541,11 @@
FileSplit[] splits = SplitsAndConstraintsUtil.getIndexSplits(appCtx.getClusterStateManager(),
primaryIndexInfo.dataset, secondaryIndex.getIndexName(), nodes);
fileSplitProvider = new ConstantFileSplitProvider(splits);
- ITypeTraits[] secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
+ secondaryIndexTypeTraits = createSecondaryIndexTypeTraits(primaryIndexInfo.recordType,
primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
- ISerializerDeserializer<?>[] secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType,
- primaryIndexInfo.metaType, primaryIndexInfo.primaryKeyTypes,
+ secondaryIndexSerdes = createSecondaryIndexSerdes(primaryIndexInfo.recordType, primaryIndexInfo.metaType,
+ primaryIndexInfo.primaryKeyTypes,
secondaryIndex.getKeyFieldTypes().toArray(new IAType[secondaryIndex.getKeyFieldTypes().size()]));
rDesc = new RecordDescriptor(secondaryIndexSerdes, secondaryIndexTypeTraits);
insertFieldsPermutations = new int[secondaryIndexTypeTraits.length];
@@ -527,6 +561,10 @@
public IFileSplitProvider getFileSplitProvider() {
return fileSplitProvider;
}
+
+ public ISerializerDeserializer<?>[] getSerdes() {
+ return secondaryIndexSerdes;
+ }
}
public static class PrimaryIndexInfo {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
index 2eba473..ff027e2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ABooleanFieldValueGenerator.java
@@ -22,7 +22,7 @@
import java.io.IOException;
import java.util.Random;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.om.types.ATypeTag;
public class ABooleanFieldValueGenerator implements IAsterixFieldValueGenerator<Boolean> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
index e698676..64dab3d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ADoubleFieldValueGenerator.java
@@ -26,7 +26,7 @@
import java.util.List;
import java.util.Random;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.om.types.ATypeTag;
public class ADoubleFieldValueGenerator implements IAsterixFieldValueGenerator<Double> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
index 7c6556b..5540c11 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt32FieldValueGenerator.java
@@ -26,7 +26,7 @@
import java.util.List;
import java.util.Random;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.om.types.ATypeTag;
public class AInt32FieldValueGenerator implements IAsterixFieldValueGenerator<Integer> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
index 2a2496e..e62054a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AInt64FieldValueGenerator.java
@@ -26,7 +26,7 @@
import java.util.List;
import java.util.Random;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.om.types.ATypeTag;
public class AInt64FieldValueGenerator implements IAsterixFieldValueGenerator<Long> {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
index cd4de62..b242189 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/ARecordValueGenerator.java
@@ -21,7 +21,7 @@
import java.io.DataOutput;
import java.io.IOException;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
index 5ee6d40..419d1b6 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/AStringFieldValueGenerator.java
@@ -26,7 +26,7 @@
import java.util.List;
import java.util.Random;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import org.apache.hyracks.util.string.UTF8StringReader;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
new file mode 100644
index 0000000..1b04c1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/RecordTupleGenerator.java
@@ -0,0 +1,126 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements. See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership. The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT 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.app.data.gen;
+
+import java.io.IOException;
+
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.test.common.TestTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class RecordTupleGenerator {
+
+ private final int[] keyIndexes;
+ private final int[] keyIndicators;
+ private final ARecordValueGenerator recordGenerator;
+ private final ARecordValueGenerator metaGenerator;
+ private final TestTupleReference tuple;
+
+ public enum GenerationFunction {
+ RANDOM,
+ DETERMINISTIC,
+ INCREASING,
+ DECREASING
+ }
+
+ /**
+ * @param recordType
+ * @param metaType
+ * @param key
+ * @param keyIndexes
+ * @param keyIndicators
+ * @param recordGeneration
+ * @param uniqueRecordFields
+ * @param metaGeneration
+ * @param uniqueMetaFields
+ */
+ public RecordTupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
+ GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
+ boolean[] uniqueMetaFields) {
+ this.keyIndexes = keyIndexes;
+ this.keyIndicators = keyIndicators;
+ for (IAType field : recordType.getFieldTypes()) {
+ validate(field);
+ }
+ recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
+ if (metaType != null) {
+ for (IAType field : metaType.getFieldTypes()) {
+ validate(field);
+ }
+ metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
+ } else {
+ metaGenerator = null;
+ }
+ int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
+ tuple = new TestTupleReference(numOfFields);
+ boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
+ for (int i = 0; i < keyIndexes.length; i++) {
+ if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
+ throw new IllegalArgumentException("key field indicator must be either 0 or 1");
+ }
+ atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
+ || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
+ keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
+ }
+ if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
+ throw new IllegalArgumentException("at least one key field must be unique and not boolean");
+ }
+ if (keyIndexes.length != keyIndicators.length) {
+ throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
+ }
+ }
+
+ private boolean validateKey(int i, ARecordType type, boolean unique) {
+ if (type.getFieldNames().length <= i) {
+ throw new IllegalArgumentException("key index must be less than number of fields");
+ }
+ return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
+ }
+
+ public ITupleReference next() throws IOException {
+ tuple.reset();
+ recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
+ if (metaGenerator != null) {
+ recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
+ }
+ for (int i = 0; i < keyIndexes.length; i++) {
+ if (keyIndicators[i] == 0) {
+ recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+ } else {
+ metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
+ }
+ }
+ return tuple;
+ }
+
+ private void validate(IAType field) {
+ switch (field.getTypeTag()) {
+ case BOOLEAN:
+ case DOUBLE:
+ case INTEGER:
+ case BIGINT:
+ case STRING:
+ break;
+ default:
+ throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
deleted file mode 100644
index 0469349..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/data/gen/TupleGenerator.java
+++ /dev/null
@@ -1,126 +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.app.data.gen;
-
-import java.io.IOException;
-
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestTupleReference;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-
-public class TupleGenerator {
-
- private final int[] keyIndexes;
- private final int[] keyIndicators;
- private final ARecordValueGenerator recordGenerator;
- private final ARecordValueGenerator metaGenerator;
- private final TestTupleReference tuple;
-
- public enum GenerationFunction {
- RANDOM,
- DETERMINISTIC,
- INCREASING,
- DECREASING
- }
-
- /**
- * @param recordType
- * @param metaType
- * @param key
- * @param keyIndexes
- * @param keyIndicators
- * @param recordGeneration
- * @param uniqueRecordFields
- * @param metaGeneration
- * @param uniqueMetaFields
- */
- public TupleGenerator(ARecordType recordType, ARecordType metaType, int[] keyIndexes, int[] keyIndicators,
- GenerationFunction[] recordGeneration, boolean[] uniqueRecordFields, GenerationFunction[] metaGeneration,
- boolean[] uniqueMetaFields) {
- this.keyIndexes = keyIndexes;
- this.keyIndicators = keyIndicators;
- for (IAType field : recordType.getFieldTypes()) {
- validate(field);
- }
- recordGenerator = new ARecordValueGenerator(recordGeneration, recordType, uniqueRecordFields, true);
- if (metaType != null) {
- for (IAType field : metaType.getFieldTypes()) {
- validate(field);
- }
- metaGenerator = new ARecordValueGenerator(metaGeneration, metaType, uniqueMetaFields, true);
- } else {
- metaGenerator = null;
- }
- int numOfFields = keyIndexes.length + 1 + ((metaType != null) ? 1 : 0);
- tuple = new TestTupleReference(numOfFields);
- boolean atLeastOneKeyFieldIsNotRandomAndNotBoolean = false;
- for (int i = 0; i < keyIndexes.length; i++) {
- if (keyIndicators[i] < 0 || keyIndicators[i] > 1) {
- throw new IllegalArgumentException("key field indicator must be either 0 or 1");
- }
- atLeastOneKeyFieldIsNotRandomAndNotBoolean = atLeastOneKeyFieldIsNotRandomAndNotBoolean
- || validateKey(keyIndexes[i], keyIndicators[i] == 0 ? recordType : metaType,
- keyIndicators[i] == 0 ? uniqueRecordFields[i] : uniqueMetaFields[i]);
- }
- if (!atLeastOneKeyFieldIsNotRandomAndNotBoolean) {
- throw new IllegalArgumentException("at least one key field must be unique and not boolean");
- }
- if (keyIndexes.length != keyIndicators.length) {
- throw new IllegalArgumentException("number of key indexes must equals number of key indicators");
- }
- }
-
- private boolean validateKey(int i, ARecordType type, boolean unique) {
- if (type.getFieldNames().length <= i) {
- throw new IllegalArgumentException("key index must be less than number of fields");
- }
- return unique && type.getFieldTypes()[i].getTypeTag() != ATypeTag.BOOLEAN;
- }
-
- public ITupleReference next() throws IOException {
- tuple.reset();
- recordGenerator.next(tuple.getFields()[keyIndexes.length].getDataOutput());
- if (metaGenerator != null) {
- recordGenerator.next(tuple.getFields()[keyIndexes.length + 1].getDataOutput());
- }
- for (int i = 0; i < keyIndexes.length; i++) {
- if (keyIndicators[i] == 0) {
- recordGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
- } else {
- metaGenerator.get(keyIndexes[i], tuple.getFields()[i].getDataOutput());
- }
- }
- return tuple;
- }
-
- private void validate(IAType field) {
- switch (field.getTypeTag()) {
- case BOOLEAN:
- case DOUBLE:
- case INTEGER:
- case BIGINT:
- case STRING:
- break;
- default:
- throw new IllegalArgumentException("Generating data of type " + field + " is not supported");
- }
- }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index a356d23..43833a2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -53,7 +53,8 @@
SIGNATURE("signature"),
STATUS("status"),
TYPE("type"),
- ERRORS("errors");
+ ERRORS("errors"),
+ PLANS("plans");
private static final Map<String, ResultField> fields = new HashMap<>();
@@ -162,6 +163,7 @@
case SIGNATURE:
case STATUS:
case TYPE:
+ case PLANS:
resultBuilder.append(OBJECT_MAPPER.writeValueAsString(fieldValue));
break;
default:
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 80048bd..f204340 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -109,6 +109,8 @@
* Static variables
*/
protected static final Logger LOGGER = LogManager.getLogger();
+ private static final String AQL = "aql";
+ private static final String SQLPP = "sqlpp";
// see
// https://stackoverflow.com/questions/417142/what-is-the-maximum-length-of-a-url-in-different-browsers/417184
private static final long MAX_URL_LENGTH = 2000l;
@@ -561,14 +563,6 @@
}
}
- public InputStream executeQuery(String str, OutputFormat fmt, URI uri, List<Parameter> params) throws Exception {
- HttpUriRequest method = constructHttpMethod(str, uri, "query", false, params);
- // Set accepted output response type
- method.setHeader("Accept", fmt.mimeType());
- HttpResponse response = executeAndCheckHttpRequest(method);
- return response.getEntity().getContent();
- }
-
public InputStream executeQueryService(String str, URI uri, OutputFormat fmt) throws Exception {
return executeQueryService(str, fmt, uri, new ArrayList<>(), false);
}
@@ -591,6 +585,10 @@
newParams = upsertParam(newParams, QueryServiceServlet.Parameter.MAX_RESULT_READS.str(),
maxReadsOptional.get());
}
+ final List<Parameter> additionalParams = extractParameters(str);
+ for (Parameter param : additionalParams) {
+ newParams = upsertParam(newParams, param.getName(), param.getValue());
+ }
HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", newParams)
: constructPostMethodUrl(str, uri, "statement", newParams);
// Set accepted output response type
@@ -626,15 +624,9 @@
}
private HttpUriRequest constructHttpMethod(String statement, URI uri, String stmtParam, boolean postStmtAsParam,
- List<Parameter> otherParams) throws URISyntaxException {
- if (statement.length() + uri.toString().length() < MAX_URL_LENGTH) {
- // Use GET for small-ish queries
- return constructGetMethod(uri, upsertParam(otherParams, stmtParam, statement));
- } else {
- // Use POST for bigger ones to avoid 413 FULL_HEAD
- String stmtParamName = (postStmtAsParam ? stmtParam : null);
- return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
- }
+ List<Parameter> otherParams) {
+ String stmtParamName = (postStmtAsParam ? stmtParam : null);
+ return constructPostMethodUrl(statement, uri, stmtParamName, otherParams);
}
private HttpUriRequest constructGetMethod(URI endpoint, List<Parameter> params) {
@@ -653,9 +645,7 @@
builder.addParameter(param.getName(), param.getValue());
}
builder.setCharset(StandardCharsets.UTF_8);
- if (body.isPresent()) {
- builder.setEntity(new StringEntity(body.get(), StandardCharsets.UTF_8));
- }
+ body.ifPresent(s -> builder.setEntity(new StringEntity(s, StandardCharsets.UTF_8)));
return builder.build();
}
@@ -683,13 +673,6 @@
return builder.build();
}
- private HttpUriRequest constructPostMethod(URI uri, OutputFormat fmt, List<Parameter> params) {
- HttpUriRequest method = constructPostMethod(uri, params);
- // Set accepted output response type
- method.setHeader("Accept", fmt.mimeType());
- return method;
- }
-
protected HttpUriRequest constructPostMethodUrl(String statement, URI uri, String stmtParam,
List<Parameter> otherParams) {
RequestBuilder builder = RequestBuilder.post(uri);
@@ -751,57 +734,6 @@
return response.getEntity().getContent();
}
- // To execute Update statements
- // Insert and Delete statements are executed here
- public void executeUpdate(String str, URI uri) throws Exception {
- // Create a method instance.
- HttpUriRequest request =
- RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
- // Execute the method.
- executeAndCheckHttpRequest(request);
- }
-
- // Executes AQL in either async or async-defer mode.
- public InputStream executeAnyAQLAsync(String statement, boolean defer, OutputFormat fmt, URI uri,
- Map<String, Object> variableCtx) throws Exception {
- // Create a method instance.
- HttpUriRequest request =
- RequestBuilder.post(uri).addParameter("mode", defer ? "asynchronous-deferred" : "asynchronous")
- .setEntity(new StringEntity(statement, StandardCharsets.UTF_8))
- .setHeader("Accept", fmt.mimeType()).build();
-
- String handleVar = getHandleVariable(statement);
-
- HttpResponse response = executeAndCheckHttpRequest(request);
- InputStream resultStream = response.getEntity().getContent();
- String resultStr = IOUtils.toString(resultStream, "UTF-8");
- ObjectNode resultJson = new ObjectMapper().readValue(resultStr, ObjectNode.class);
- final JsonNode jsonHandle = resultJson.get("handle");
- final String strHandle = jsonHandle.asText();
-
- if (handleVar != null) {
- variableCtx.put(handleVar, strHandle);
- return resultStream;
- }
- return null;
- }
-
- // To execute DDL and Update statements
- // create type statement
- // create dataset statement
- // create index statement
- // create dataverse statement
- // create function statement
- public void executeDDL(String str, URI uri) throws Exception {
- // Create a method instance.
- HttpUriRequest request =
- RequestBuilder.post(uri).setEntity(new StringEntity(str, StandardCharsets.UTF_8)).build();
-
- // Execute the method.
- executeAndCheckHttpRequest(request);
- }
-
// Method that reads a DDL/Update/Query File
// and returns the contents as a string
// This string is later passed to REST API for execution.
@@ -886,13 +818,15 @@
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath)
throws Exception {
+ URI uri;
+ InputStream resultStream;
File qbcFile;
boolean failed = false;
File expectedResultFile;
switch (ctx.getType()) {
case "ddl":
if (ctx.getFile().getName().endsWith("aql")) {
- executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+ executeAqlUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
} else {
executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
}
@@ -903,7 +837,7 @@
statement = statement.replaceAll("nc1://", "127.0.0.1://../../../../../../asterix-app/");
}
if (ctx.getFile().getName().endsWith("aql")) {
- executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+ executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
} else {
executeSqlppUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
}
@@ -957,14 +891,12 @@
expectedResultFileCtxs);
break;
case "txnqbc": // qbc represents query before crash
- InputStream resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
- getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+ resultStream = query(cUnit, testFile.getName(), statement);
qbcFile = getTestCaseQueryBeforeCrashFile(actualPath, testCaseCtx, cUnit);
writeOutputToFile(qbcFile, resultStream);
break;
case "txnqar": // qar represents query after recovery
- resultStream = executeQuery(statement, OutputFormat.forCompilationUnit(cUnit),
- getEndpoint(Servlets.AQL_QUERY), cUnit.getParameter());
+ resultStream = query(cUnit, testFile.getName(), statement);
File qarFile = new File(actualPath + File.separator
+ testCaseCtx.getTestCase().getFilePath().replace(File.separator, "_") + "_" + cUnit.getName()
+ "_qar.adm");
@@ -974,7 +906,7 @@
break;
case "txneu": // eu represents erroneous update
try {
- executeUpdate(statement, getEndpoint(Servlets.AQL_UPDATE));
+ executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
} catch (Exception e) {
// An exception is expected.
failed = true;
@@ -1001,7 +933,7 @@
break;
case "errddl": // a ddlquery that expects error
try {
- executeDDL(statement, getEndpoint(Servlets.AQL_DDL));
+ executeAqlUpdateOrDdl(statement, OutputFormat.forCompilationUnit(cUnit));
} catch (Exception e) {
// expected error happens
failed = true;
@@ -1238,38 +1170,25 @@
public void executeQuery(OutputFormat fmt, String statement, Map<String, Object> variableCtx, String reqType,
File testFile, File expectedResultFile, File actualResultFile, MutableInt queryCount, int numResultFiles,
List<Parameter> params, ComparisonEnum compare) throws Exception {
- InputStream resultStream = null;
- if (testFile.getName().endsWith("aql")) {
- if (reqType.equalsIgnoreCase("query")) {
- resultStream = executeQuery(statement, fmt, getEndpoint(Servlets.AQL_QUERY), params);
- } else {
- final URI endpoint = getEndpoint(Servlets.AQL);
- if (reqType.equalsIgnoreCase("async")) {
- resultStream = executeAnyAQLAsync(statement, false, fmt, endpoint, variableCtx);
- } else if (reqType.equalsIgnoreCase("deferred")) {
- resultStream = executeAnyAQLAsync(statement, true, fmt, endpoint, variableCtx);
- }
- Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), resultStream);
- }
+ String delivery = DELIVERY_IMMEDIATE;
+ if (reqType.equalsIgnoreCase("async")) {
+ delivery = DELIVERY_ASYNC;
+ } else if (reqType.equalsIgnoreCase("deferred")) {
+ delivery = DELIVERY_DEFERRED;
+ }
+ URI uri = testFile.getName().endsWith("aql") ? getEndpoint(Servlets.QUERY_AQL)
+ : getEndpoint(Servlets.QUERY_SERVICE);
+ InputStream resultStream;
+ if (DELIVERY_IMMEDIATE.equals(delivery)) {
+ resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
+ resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
+ : ResultExtractor.extract(resultStream);
} else {
- String delivery = DELIVERY_IMMEDIATE;
- if (reqType.equalsIgnoreCase("async")) {
- delivery = DELIVERY_ASYNC;
- } else if (reqType.equalsIgnoreCase("deferred")) {
- delivery = DELIVERY_DEFERRED;
- }
- final URI uri = getEndpoint(Servlets.QUERY_SERVICE);
- if (DELIVERY_IMMEDIATE.equals(delivery)) {
- resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
- resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
- : ResultExtractor.extract(resultStream);
- } else {
- String handleVar = getHandleVariable(statement);
- resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
- String handle = ResultExtractor.extractHandle(resultStream);
- Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
- variableCtx.put(handleVar, handle);
- }
+ String handleVar = getHandleVariable(statement);
+ resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
+ String handle = ResultExtractor.extractHandle(resultStream);
+ Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
+ variableCtx.put(handleVar, toQueryServiceHandle(handle));
}
if (actualResultFile == null) {
if (testFile.getName().startsWith(DIAGNOSE)) {
@@ -1440,7 +1359,16 @@
}
public InputStream executeSqlppUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
- InputStream resultStream = executeQueryService(statement, getEndpoint(Servlets.QUERY_SERVICE), outputFormat);
+ return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP));
+ }
+
+ private InputStream executeAqlUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
+ return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(AQL));
+ }
+
+ private InputStream executeUpdateOrDdl(String statement, OutputFormat outputFormat, URI serviceUri)
+ throws Exception {
+ InputStream resultStream = executeQueryService(statement, serviceUri, outputFormat);
return ResultExtractor.extract(resultStream);
}
@@ -1947,4 +1875,19 @@
private static boolean isCancellable(String type) {
return !NON_CANCELLABLE.contains(type);
}
+
+ private InputStream query(CompilationUnit cUnit, String testFile, String statement) throws Exception {
+ final URI uri = getQueryServiceUri(testFile);
+ final InputStream inputStream = executeQueryService(statement, OutputFormat.forCompilationUnit(cUnit), uri,
+ cUnit.getParameter(), true, null, false);
+ return ResultExtractor.extract(inputStream);
+ }
+
+ private URI getQueryServiceUri(String extension) throws URISyntaxException {
+ return extension.endsWith(AQL) ? getEndpoint(Servlets.QUERY_AQL) : getEndpoint(Servlets.QUERY_SERVICE);
+ }
+
+ private static String toQueryServiceHandle(String handle) {
+ return handle.replace("/aql/", "/service/");
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
new file mode 100644
index 0000000..c34f5a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt32ValueGenerator.java
@@ -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.
+ */
+package org.apache.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt32;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt32ValueGenerator implements IFieldValueGenerator<AInt32> {
+ int counter = 0;
+
+ @Override
+ public AInt32 next() {
+ return new AInt32(counter++);
+ }
+
+ @Override
+ public void reset() {
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
new file mode 100644
index 0000000..b860737
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/AInt64ValueGenerator.java
@@ -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.
+ */
+package org.apache.asterix.test.dataflow;
+
+import org.apache.asterix.om.base.AInt64;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+
+public class AInt64ValueGenerator implements IFieldValueGenerator<AInt64> {
+ long counter = 0L;
+
+ @Override
+ public AInt64 next() {
+ return new AInt64(counter++);
+ }
+
+ @Override
+ public void reset() {
+ }
+
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
new file mode 100644
index 0000000..a0ed26e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/CheckpointInSecondaryIndexTest.java
@@ -0,0 +1,577 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.dataflow;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.app.bootstrap.TestNodeController;
+import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
+import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.config.DatasetConfig.IndexType;
+import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.IndexCheckpoint;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionManager;
+import org.apache.asterix.common.transactions.TransactionOptions;
+import org.apache.asterix.external.util.DataflowUtils;
+import org.apache.asterix.file.StorageComponentProvider;
+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.entities.InternalDatasetDetails.PartitioningStrategy;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorNodePushable;
+import org.apache.asterix.test.common.TestHelper;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
+import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
+import org.apache.hyracks.storage.am.common.datagen.IFieldValueGenerator;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+public class CheckpointInSecondaryIndexTest {
+ static final int REPREAT_TEST_COUNT = 1;
+
+ @Parameterized.Parameters
+ public static List<Object[]> data() {
+ return Arrays.asList(new Object[REPREAT_TEST_COUNT][0]);
+ }
+
+ private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
+ private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
+ new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
+ private static final GenerationFunction[] RECORD_GEN_FUNCTION =
+ { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
+ private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
+ private static final ARecordType META_TYPE = null;
+ private static final GenerationFunction[] META_GEN_FUNCTION = null;
+ private static final boolean[] UNIQUE_META_FIELDS = null;
+ private static final int[] KEY_INDEXES = { 0 };
+ private static final int[] KEY_INDICATORS = { Index.RECORD_INDICATOR };
+ private static final List<Integer> KEY_INDICATORS_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
+ private static final int RECORDS_PER_COMPONENT = 500;
+ private static final int DATASET_ID = 101;
+ private static final String DATAVERSE_NAME = "TestDV";
+ private static final String DATASET_NAME = "TestDS";
+ private static final String INDEX_NAME = "TestIdx";
+ private static final String DATA_TYPE_NAME = "DUMMY";
+ private static final String NODE_GROUP_NAME = "DEFAULT";
+ private static final IndexType INDEX_TYPE = IndexType.BTREE;
+ private static final IFieldValueGenerator[] SECONDARY_INDEX_VALUE_GENERATOR =
+ { new AInt64ValueGenerator(), new AInt32ValueGenerator() };
+ private static final List<List<String>> INDEX_FIELD_NAMES =
+ Arrays.asList(Arrays.asList(RECORD_TYPE.getFieldNames()[1]));
+ private static final List<Integer> INDEX_FIELD_INDICATORS = Arrays.asList(Index.RECORD_INDICATOR);
+ private static final List<IAType> INDEX_FIELD_TYPES = Arrays.asList(BuiltinType.AINT64);
+ private static final StorageComponentProvider storageManager = new StorageComponentProvider();
+ private static TestNodeController nc;
+ private static NCAppRuntimeContext ncAppCtx;
+ private static IDatasetLifecycleManager dsLifecycleMgr;
+ private static Dataset dataset;
+ private static Index secondaryIndex;
+ private static ITransactionContext txnCtx;
+ private static TestLsmBtree primaryLsmBtree;
+ private static TestLsmBtree secondaryLsmBtree;
+ private static PrimaryIndexInfo primaryIndexInfo;
+ private static IHyracksTaskContext taskCtx;
+ private static IIndexDataflowHelper primaryIndexDataflowHelper;
+ private static IIndexDataflowHelper secondaryIndexDataflowHelper;
+ private static LSMInsertDeleteOperatorNodePushable insertOp;
+ private static LSMIndexBulkLoadOperatorNodePushable indexLoadOp;
+ private static IHyracksTaskContext loadTaskCtx;
+ private static SecondaryIndexInfo secondaryIndexInfo;
+ private static Actor actor;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ System.out.println("SetUp: ");
+ TestHelper.deleteExistingInstanceFiles();
+ String configPath = System.getProperty("user.dir") + File.separator + "src" + File.separator + "test"
+ + File.separator + "resources" + File.separator + "cc-multipart.conf";
+ nc = new TestNodeController(configPath, false);
+ nc.init();
+ ncAppCtx = nc.getAppRuntimeContext();
+ dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ System.out.println("TearDown");
+ nc.deInit();
+ TestHelper.deleteExistingInstanceFiles();
+ }
+
+ @Before
+ public void createIndex() throws Exception {
+ List<List<String>> partitioningKeys = new ArrayList<>();
+ partitioningKeys.add(Collections.singletonList("key"));
+ dataset = new TestDataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME,
+ NODE_GROUP_NAME, NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null,
+ PartitioningStrategy.HASH, partitioningKeys, null, null, null, false, null),
+ null, DatasetType.INTERNAL, DATASET_ID, 0);
+ secondaryIndex = new Index(DATAVERSE_NAME, DATASET_NAME, INDEX_NAME, INDEX_TYPE, INDEX_FIELD_NAMES,
+ INDEX_FIELD_INDICATORS, INDEX_FIELD_TYPES, false, false, false, 0);
+ taskCtx = null;
+ primaryIndexDataflowHelper = null;
+ secondaryIndexDataflowHelper = null;
+ primaryLsmBtree = null;
+ insertOp = null;
+ JobId jobId = nc.newJobId();
+ txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(jobId),
+ new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
+ actor = null;
+ taskCtx = nc.createTestContext(jobId, 0, false);
+ primaryIndexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+ KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+ IndexDataflowHelperFactory iHelperFactory =
+ new IndexDataflowHelperFactory(nc.getStorageManager(), primaryIndexInfo.getFileSplitProvider());
+ primaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+ primaryIndexDataflowHelper.open();
+ primaryLsmBtree = (TestLsmBtree) primaryIndexDataflowHelper.getIndexInstance();
+ primaryIndexDataflowHelper.close();
+ // This pipeline skips the secondary index
+ insertOp = nc.getInsertPipeline(taskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+ KEY_INDICATORS_LIST, storageManager, null).getLeft();
+ actor = new Actor("player");
+ // allow all operations
+ StorageTestUtils.allowAllOps(primaryLsmBtree);
+ actor.add(new Request(Request.Action.INSERT_OPEN));
+ }
+
+ @After
+ public void destroyIndex() throws Exception {
+ Request close = new Request(Request.Action.INSERT_CLOSE);
+ actor.add(close);
+ close.await();
+ nc.getTransactionManager().commitTransaction(txnCtx.getTxnId());
+ if (secondaryIndexDataflowHelper != null) {
+ secondaryIndexDataflowHelper.destroy();
+ }
+ primaryIndexDataflowHelper.destroy();
+ actor.stop();
+ }
+
+ @Test
+ public void testCheckpointUpdatedWhenSecondaryIsEmpty() throws Exception {
+ try {
+ // create secondary
+ createSecondaryIndex();
+ actor.add(new Request(Request.Action.INSERT_PATCH));
+ ensureDone(actor);
+ // search now and ensure partition 0 has all the records
+ StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+ // and that secondary index is empty
+ Assert.assertTrue(secondaryLsmBtree.isCurrentMutableComponentEmpty());
+ // flush
+ actor.add(new Request(Request.Action.FLUSH_DATASET));
+ ensureDone(actor);
+ // ensure primary has a component
+ Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+
+ // ensure secondary doesn't have a component
+ Assert.assertEquals(0, secondaryLsmBtree.getDiskComponents().size());
+ // ensure that current memory component index match
+ Assert.assertEquals(secondaryLsmBtree.getCurrentMemoryComponentIndex(),
+ primaryLsmBtree.getCurrentMemoryComponentIndex());
+ // ensure both checkpoint files has the same component id as the last flushed component id
+ ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+ LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+ long min = id.getMinId();
+ // primary ref
+ Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+ fileManagerField.setAccessible(true); //Make it accessible so you can access it
+ ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+ final ResourceReference primaryRef = ResourceReference
+ .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+ // secondary ref
+ ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+ final ResourceReference secondaryRef = ResourceReference.of(
+ secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+ IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+ } catch (Throwable e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ private void createSecondaryIndex()
+ throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+ SecondaryIndexInfo secondaryIndexInfo =
+ nc.createSecondaryIndex(primaryIndexInfo, secondaryIndex, storageManager, 0);
+ IndexDataflowHelperFactory iHelperFactory =
+ new IndexDataflowHelperFactory(nc.getStorageManager(), secondaryIndexInfo.getFileSplitProvider());
+ secondaryIndexDataflowHelper = iHelperFactory.create(taskCtx.getJobletContext().getServiceContext(), 0);
+ secondaryIndexDataflowHelper.open();
+ secondaryLsmBtree = (TestLsmBtree) secondaryIndexDataflowHelper.getIndexInstance();
+ secondaryIndexDataflowHelper.close();
+ }
+
+ @Test
+ public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsSingleComponent() throws Exception {
+ try {
+ // create secondary
+ actor.add(new Request(Request.Action.INSERT_PATCH));
+ ensureDone(actor);
+ // search now and ensure partition 0 has all the records
+ StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+ // flush
+ actor.add(new Request(Request.Action.FLUSH_DATASET));
+ ensureDone(actor);
+ // ensure primary has a component
+ Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+ // ensure both checkpoint files has the same component id as the last flushed component id
+ ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+ LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+ long min = id.getMinId();
+ // primary ref
+ Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+ fileManagerField.setAccessible(true); //Make it accessible so you can access it
+ ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+ final ResourceReference primaryRef = ResourceReference
+ .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+ IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ createSecondaryIndex();
+ JobId jobId = nc.newJobId();
+ loadTaskCtx = nc.createTestContext(jobId, 0, false);
+ Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+ nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+ indexLoadOp = infoAndOp.getRight();
+ secondaryIndexInfo = infoAndOp.getLeft();
+ actor.add(new Request(Request.Action.LOAD_OPEN));
+ actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+ actor.add(new Request(Request.Action.LOAD_CLOSE));
+ ensureDone(actor);
+ latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+ final ResourceReference secondaryRef = ResourceReference.of(
+ secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+ IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+ } catch (Throwable e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ @Test
+ public void testCheckpointWhenBulkloadingSecondaryAndPrimaryIsTwoComponents() throws Exception {
+ try {
+ // create secondary
+ actor.add(new Request(Request.Action.INSERT_PATCH));
+ ensureDone(actor);
+ // search now and ensure partition 0 has all the records
+ StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+ // flush
+ actor.add(new Request(Request.Action.FLUSH_DATASET));
+ ensureDone(actor);
+ // ensure primary has a component
+ Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+ // ensure both checkpoint files has the same component id as the last flushed component id
+ ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+ LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+ long min = id.getMinId();
+ // primary ref
+ Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+ fileManagerField.setAccessible(true); //Make it accessible so you can access it
+ ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+ final ResourceReference primaryRef = ResourceReference
+ .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+ IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ actor.add(new Request(Request.Action.INSERT_PATCH));
+ ensureDone(actor);
+ actor.add(new Request(Request.Action.FLUSH_DATASET));
+ ensureDone(actor);
+ Assert.assertEquals(2, primaryLsmBtree.getDiskComponents().size());
+ // ensure both checkpoint files has the same component id as the last flushed component id
+ primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+ id = (LSMComponentId) primaryDiskComponent.getId();
+ min = id.getMaxId();
+ createSecondaryIndex();
+ JobId jobId = nc.newJobId();
+ loadTaskCtx = nc.createTestContext(jobId, 0, false);
+ Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+ nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+ indexLoadOp = infoAndOp.getRight();
+ secondaryIndexInfo = infoAndOp.getLeft();
+ actor.add(new Request(Request.Action.LOAD_OPEN));
+ actor.add(new Request(Request.Action.INDEX_LOAD_PATCH));
+ actor.add(new Request(Request.Action.LOAD_CLOSE));
+ ensureDone(actor);
+ latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+ final ResourceReference secondaryRef = ResourceReference.of(
+ secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+ IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+ } catch (Throwable e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ @Test
+ public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsEmpty() throws Exception {
+ try {
+ // ensure primary has no component
+ Assert.assertEquals(0, primaryLsmBtree.getDiskComponents().size());
+ // primary ref
+ Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+ fileManagerField.setAccessible(true); //Make it accessible so you can access it
+ ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+ final ResourceReference primaryRef = ResourceReference
+ .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+ IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ createSecondaryIndex();
+ JobId jobId = nc.newJobId();
+ loadTaskCtx = nc.createTestContext(jobId, 0, false);
+ Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+ nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+ indexLoadOp = infoAndOp.getRight();
+ secondaryIndexInfo = infoAndOp.getLeft();
+ actor.add(new Request(Request.Action.LOAD_OPEN));
+ actor.add(new Request(Request.Action.LOAD_CLOSE));
+ ensureDone(actor);
+ latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+ final ResourceReference secondaryRef = ResourceReference.of(
+ secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+ IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(),
+ latestPrimaryCheckpoint.getLastComponentId());
+ } catch (Throwable e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ @Test
+ public void testCheckpointWhenBulkloadedSecondaryIsEmptyAndPrimaryIsNotEmpty() throws Exception {
+ try {
+ // create secondary
+ actor.add(new Request(Request.Action.INSERT_PATCH));
+ ensureDone(actor);
+ // search now and ensure partition 0 has all the records
+ StorageTestUtils.searchAndAssertCount(nc, 0, dataset, storageManager, RECORDS_PER_COMPONENT);
+ // flush
+ actor.add(new Request(Request.Action.FLUSH_DATASET));
+ ensureDone(actor);
+ // ensure primary has a component
+ Assert.assertEquals(1, primaryLsmBtree.getDiskComponents().size());
+ // ensure both checkpoint files has the same component id as the last flushed component id
+ ILSMDiskComponent primaryDiskComponent = primaryLsmBtree.getDiskComponents().get(0);
+ LSMComponentId id = (LSMComponentId) primaryDiskComponent.getId();
+ long min = id.getMinId();
+ // primary ref
+ Field fileManagerField = AbstractLSMIndex.class.getDeclaredField("fileManager"); //get private declared object from class
+ fileManagerField.setAccessible(true); //Make it accessible so you can access it
+ ILSMIndexFileManager primaryFileManager = (ILSMIndexFileManager) fileManagerField.get(primaryLsmBtree);
+ final ResourceReference primaryRef = ResourceReference
+ .of(primaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager primaryCheckpointManager = getIndexCheckpointManagerProvider().get(primaryRef);
+ IndexCheckpoint latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ createSecondaryIndex();
+ JobId jobId = nc.newJobId();
+ loadTaskCtx = nc.createTestContext(jobId, 0, false);
+ Pair<SecondaryIndexInfo, LSMIndexBulkLoadOperatorNodePushable> infoAndOp =
+ nc.getBulkLoadSecondaryOperator(loadTaskCtx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, secondaryIndex, RECORDS_PER_COMPONENT);
+ indexLoadOp = infoAndOp.getRight();
+ secondaryIndexInfo = infoAndOp.getLeft();
+ actor.add(new Request(Request.Action.LOAD_OPEN));
+ actor.add(new Request(Request.Action.LOAD_CLOSE));
+ ensureDone(actor);
+ latestPrimaryCheckpoint = primaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestPrimaryCheckpoint.getLastComponentId(), min);
+ ILSMIndexFileManager secondaryFileManager = (ILSMIndexFileManager) fileManagerField.get(secondaryLsmBtree);
+ final ResourceReference secondaryRef = ResourceReference.of(
+ secondaryFileManager.getRelFlushFileReference().getInsertIndexFileReference().getAbsolutePath());
+ IIndexCheckpointManager secondaryCheckpointManager = getIndexCheckpointManagerProvider().get(secondaryRef);
+ IndexCheckpoint latestSecondaryCheckpoint = secondaryCheckpointManager.getLatest();
+ Assert.assertEquals(latestSecondaryCheckpoint.getLastComponentId(), min);
+ } catch (Throwable e) {
+ e.printStackTrace();
+ throw e;
+ }
+ }
+
+ protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+ return ncAppCtx.getIndexCheckpointManagerProvider();
+ }
+
+ private void ensureDone(Actor actor) throws InterruptedException {
+ Request req = new Request(Request.Action.DUMMY);
+ actor.add(req);
+ req.await();
+ }
+
+ private static class Request {
+ enum Action {
+ DUMMY,
+ INSERT_OPEN,
+ LOAD_OPEN,
+ INSERT_PATCH,
+ INDEX_LOAD_PATCH,
+ FLUSH_DATASET,
+ INSERT_CLOSE,
+ LOAD_CLOSE,
+ }
+
+ private final Action action;
+ private volatile boolean done;
+
+ public Request(Action action) {
+ this.action = action;
+ done = false;
+ }
+
+ synchronized void complete() {
+ done = true;
+ notifyAll();
+ }
+
+ synchronized void await() throws InterruptedException {
+ while (!done) {
+ wait();
+ }
+ }
+ }
+
+ public class Actor extends SingleThreadEventProcessor<Request> {
+ private final RecordTupleGenerator primaryInsertTupleGenerator;
+ private final FrameTupleAppender tupleAppender;
+
+ public Actor(String name) throws HyracksDataException {
+ super(name);
+ primaryInsertTupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+ RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ tupleAppender = new FrameTupleAppender(new VSizeFrame(taskCtx));
+ }
+
+ @Override
+ protected void handle(Request req) throws Exception {
+ try {
+ switch (req.action) {
+ case FLUSH_DATASET:
+ if (tupleAppender.getTupleCount() > 0) {
+ tupleAppender.write(insertOp, true);
+ }
+ dsLifecycleMgr.flushDataset(dataset.getDatasetId(), false);
+ break;
+ case INSERT_CLOSE:
+ insertOp.close();
+ break;
+ case INSERT_OPEN:
+ insertOp.open();
+ break;
+ case LOAD_OPEN:
+ indexLoadOp.open();
+ break;
+ case LOAD_CLOSE:
+ indexLoadOp.close();
+ break;
+ case INSERT_PATCH:
+ for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+ ITupleReference tuple = primaryInsertTupleGenerator.next();
+ DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
+ }
+ if (tupleAppender.getTupleCount() > 0) {
+ tupleAppender.write(insertOp, true);
+ }
+ StorageTestUtils.waitForOperations(primaryLsmBtree);
+ break;
+ case INDEX_LOAD_PATCH:
+ TupleGenerator secondaryLoadTupleGenerator =
+ new TupleGenerator(SECONDARY_INDEX_VALUE_GENERATOR, secondaryIndexInfo.getSerdes(), 0);
+ FrameTupleAppender secondaryTupleAppender = new FrameTupleAppender(new VSizeFrame(loadTaskCtx));
+ for (int j = 0; j < RECORDS_PER_COMPONENT; j++) {
+ ITupleReference tuple = secondaryLoadTupleGenerator.next();
+ DataflowUtils.addTupleToFrame(secondaryTupleAppender, tuple, indexLoadOp);
+ }
+ if (secondaryTupleAppender.getTupleCount() > 0) {
+ secondaryTupleAppender.write(indexLoadOp, true);
+ }
+ break;
+ default:
+ break;
+ }
+ } catch (Throwable th) {
+ th.printStackTrace();
+ throw th;
+ } finally {
+ req.complete();
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
index 9ef531e..017c59f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/ComponentRollbackTest.java
@@ -20,16 +20,18 @@
import java.io.File;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.function.Predicate;
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.transactions.ITransactionContext;
import org.apache.asterix.common.transactions.ITransactionManager;
import org.apache.asterix.common.transactions.TransactionOptions;
@@ -50,6 +52,7 @@
import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
@@ -119,7 +122,7 @@
// allow all operations
StorageTestUtils.allowAllOps(lsmBtree);
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -146,16 +149,27 @@
StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ ILSMComponentId next =
+ dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
// rollback a memory component
lsmAccessor.deleteComponents(memoryComponentsPredicate);
StorageTestUtils.searchAndAssertCount(nc, PARTITION,
StorageTestUtils.TOTAL_NUM_OF_RECORDS - StorageTestUtils.RECORDS_PER_COMPONENT);
// rollback the last disk component
lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
lsmAccessor.deleteComponents(pred);
StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -176,7 +190,7 @@
// allow all operations
StorageTestUtils.allowAllOps(lsmBtree);
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -203,7 +217,13 @@
StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ ILSMComponentId next =
+ dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
// rollback a memory component
lsmAccessor.deleteComponents(memoryComponentsPredicate);
StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -227,9 +247,14 @@
StorageTestUtils.searchAndAssertCount(nc, PARTITION, StorageTestUtils.TOTAL_NUM_OF_RECORDS);
// rollback the last disk component
lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
lsmAccessor.deleteComponents(pred);
StorageTestUtils.searchAndAssertCount(nc, PARTITION,
@@ -247,7 +272,7 @@
StorageTestUtils.allowAllOps(lsmBtree);
lsmBtree.clearSearchCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -278,7 +303,13 @@
// now that we enetered, we will rollback
ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ ILSMComponentId next =
+ dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
// rollback a memory component
lsmAccessor.deleteComponents(
c -> (c instanceof ILSMMemoryComponent && ((ILSMMemoryComponent) c).isModified()));
@@ -297,10 +328,14 @@
// wait till firstSearcher enter the components
secondSearcher.waitUntilEntered();
lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
-
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ next = dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
DiskComponentLsnPredicate pred = new DiskComponentLsnPredicate(lsn);
lsmAccessor.deleteComponents(pred);
// now that the rollback has completed, we will unblock the search
@@ -321,7 +356,7 @@
// allow all operations
StorageTestUtils.allowAllOps(lsmBtree);
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -376,7 +411,7 @@
StorageTestUtils.allowAllOps(lsmBtree);
lsmBtree.clearMergeCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -407,11 +442,11 @@
// select the components to merge... the last three
int numMergedComponents = 3;
List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
for (int i = 0; i < numMergedComponents; i++) {
mergedComponents.add(diskComponents.get(i));
}
- mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+ mergeAccessor.scheduleMerge(mergedComponents);
merger.waitUntilCount(1);
// now that we enetered, we will rollback
Rollerback rollerback = new Rollerback(lsmBtree, new DiskComponentLsnPredicate(lsn));
@@ -439,7 +474,7 @@
// allow all operations
StorageTestUtils.allowAllOps(lsmBtree);
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -498,7 +533,7 @@
// allow all operations
StorageTestUtils.allowAllOps(lsmBtree);
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -559,7 +594,7 @@
StorageTestUtils.allowAllOps(lsmBtree);
lsmBtree.clearMergeCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -590,11 +625,11 @@
// select the components to merge... the last three
int numMergedComponents = 3;
List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
for (int i = 0; i < numMergedComponents; i++) {
mergedComponents.add(diskComponents.get(i));
}
- mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+ mergeAccessor.scheduleMerge(mergedComponents);
merger.waitUntilCount(1);
// we will block search
lsmBtree.clearSearchCallbacks();
@@ -629,7 +664,7 @@
StorageTestUtils.allowAllOps(lsmBtree);
lsmBtree.clearMergeCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
for (int j = 0; j < StorageTestUtils.TOTAL_NUM_OF_RECORDS; j++) {
@@ -659,12 +694,12 @@
ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
// select the components to merge... the last three
List<ILSMDiskComponent> mergedComponents = new ArrayList<>();
- long lsn = AbstractLSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
+ long lsn = LSMIOOperationCallback.getTreeIndexLSN(diskComponents.get(0).getMetadata());
int numMergedComponents = 3;
for (int i = 0; i < numMergedComponents; i++) {
mergedComponents.add(diskComponents.get(i));
}
- mergeAccessor.scheduleMerge(lsmBtree.getIOOperationCallback(), mergedComponents);
+ mergeAccessor.scheduleMerge(mergedComponents);
merger.waitUntilCount(1);
// we will block search
lsmBtree.clearSearchCallbacks();
@@ -704,7 +739,13 @@
ILSMIndexAccessor lsmAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
try {
dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).refresh();
- ((AbstractLSMIOOperationCallback) lsmBtree.getIOOperationCallback()).updateLastLSN(0);
+ ILSMComponentId next =
+ dsLifecycleMgr.getComponentIdGenerator(StorageTestUtils.DATASET_ID, PARTITION).getId();
+ long flushLsn = nc.getTransactionSubsystem().getLogManager().getAppendLSN();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, next);
+ lsmAccessor.getOpContext().setParameters(flushMap);
lsmAccessor.deleteComponents(predicate);
} catch (HyracksDataException e) {
failure = e;
@@ -733,9 +774,8 @@
@Override
public boolean test(ILSMComponent c) {
try {
- return c instanceof ILSMMemoryComponent
- || (c instanceof ILSMDiskComponent && AbstractLSMIOOperationCallback
- .getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
+ return c instanceof ILSMMemoryComponent || (c instanceof ILSMDiskComponent
+ && LSMIOOperationCallback.getTreeIndexLSN(((ILSMDiskComponent) c).getMetadata()) >= lsn);
} catch (HyracksDataException e) {
e.printStackTrace();
return false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
index 8bafd32..b618727 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/IoCallbackFailureTest.java
@@ -22,7 +22,7 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -91,7 +91,7 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(ILSMMemoryComponent c) throws HyracksDataException {
// No Op
}
};
@@ -118,7 +118,7 @@
throws Exception {
NCAppRuntimeContext ncAppCtx = nc.getAppRuntimeContext();
IDatasetLifecycleManager dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
- TupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
+ RecordTupleGenerator tupleGenerator = StorageTestUtils.getTupleGenerator();
ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
boolean failed = false;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index dabb32c..79e6368 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -19,6 +19,7 @@
package org.apache.asterix.test.dataflow;
import java.io.File;
+import java.lang.reflect.Field;
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Arrays;
@@ -28,7 +29,7 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
@@ -60,8 +61,14 @@
import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -69,6 +76,7 @@
import org.junit.Test;
public class LSMFlushRecoveryTest {
+ public static final Logger LOGGER = LogManager.getLogger();
private static TestNodeController nc;
private static Dataset dataset;
private static PrimaryIndexInfo[] primaryIndexInfos;
@@ -84,7 +92,7 @@
private static IIndexDataflowHelper[] secondaryIndexDataflowHelpers;
private static ITransactionContext txnCtx;
private static LSMInsertDeleteOperatorNodePushable[] insertOps;
- private static TupleGenerator tupleGenerator;
+ private static RecordTupleGenerator tupleGenerator;
private static final int NUM_PARTITIONS = 2;
private static final int PARTITION_0 = 0;
@@ -153,6 +161,22 @@
private void initializeNc(boolean cleanUpOnStart) throws Exception {
nc.init(cleanUpOnStart);
ncAppCtx = nc.getAppRuntimeContext();
+ // Override the LSMIOScheduler to avoid halting on failure and enable
+ // testing failure scenario in a unit test setting
+ Field ioScheduler = ncAppCtx.getClass().getDeclaredField("lsmIOScheduler");
+ ioScheduler.setAccessible(true);
+ ioScheduler.set(ncAppCtx, new AsynchronousScheduler(ncAppCtx.getServiceContext().getThreadFactory(),
+ new IIoOperationFailedCallback() {
+ @Override
+ public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+ LOGGER.error("Scheduler Failed", failure);
+ }
+
+ @Override
+ public void operationFailed(ILSMIOOperation operation, Throwable t) {
+ LOGGER.warn("IO Operation failed", t);
+ }
+ }));
dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
}
@@ -241,21 +265,21 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
});
Semaphore primaryFlushSemaphore = new Semaphore(0);
primaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
- primaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+ primaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
@Override
public void before(Void t) throws HyracksDataException {
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Void t) throws HyracksDataException {
primaryFlushSemaphore.release();
}
});
@@ -283,21 +307,21 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
});
Semaphore secondaryFlushSemaphore = new Semaphore(0);
secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
- secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+ secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
@Override
public void before(Void t) throws HyracksDataException {
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Void t) throws HyracksDataException {
secondaryFlushSemaphore.release();
}
});
@@ -335,21 +359,21 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
});
Semaphore flushSemaphore = new Semaphore(0);
secondaryIndexes[PARTITION_0].addFlushCallback(AllowTestOpCallback.INSTANCE);
- secondaryIndexes[PARTITION_0].addIoAfterFinalizeCallback(new ITestOpCallback<Void>() {
+ secondaryIndexes[PARTITION_0].addIoCompletedCallback(new ITestOpCallback<Void>() {
@Override
public void before(Void t) throws HyracksDataException {
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Void t) throws HyracksDataException {
flushSemaphore.release();
}
});
@@ -387,7 +411,7 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
});
@@ -400,7 +424,7 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
});
@@ -454,6 +478,8 @@
ILSMMemoryComponent primaryMemComponent = primaryIndexes[partitionIndex].getCurrentMemoryComponent();
ILSMMemoryComponent secondaryMemComponent = secondaryIndexes[partitionIndex].getCurrentMemoryComponent();
Assert.assertEquals(primaryMemComponent.getId(), secondaryMemComponent.getId());
+ Assert.assertEquals(primaryIndexes[partitionIndex].getCurrentMemoryComponentIndex(),
+ secondaryIndexes[partitionIndex].getCurrentMemoryComponentIndex());
}
List<ILSMDiskComponent> primaryDiskComponents = primaryIndexes[partitionIndex].getDiskComponents();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
index 2121327..e2c99b0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LogMarkerTest.java
@@ -26,10 +26,9 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
import org.apache.asterix.common.transactions.ILogRecord;
import org.apache.asterix.common.transactions.ITransactionContext;
@@ -37,10 +36,7 @@
import org.apache.asterix.common.transactions.TransactionOptions;
import org.apache.asterix.external.util.DataflowUtils;
import org.apache.asterix.file.StorageComponentProvider;
-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.entities.InternalDatasetDetails.PartitioningStrategy;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
@@ -113,22 +109,20 @@
StorageComponentProvider storageManager = new StorageComponentProvider();
List<List<String>> partitioningKeys = new ArrayList<>();
partitioningKeys.add(Collections.singletonList("key"));
- Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
- NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
- partitioningKeys, null, null, null, false, null),
- null, DatasetType.INTERNAL, DATASET_ID, 0);
try {
- PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
- storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
+ PrimaryIndexInfo indexInfo = nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE,
+ META_TYPE, null, storageManager, KEY_INDEXES, KEY_INDICATORS_LIST, 0);
JobId jobId = nc.newJobId();
IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, true);
ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
- LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
- RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
+ LSMInsertDeleteOperatorNodePushable insertOp =
+ nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATORS_LIST, storageManager, null).getLeft();
insertOp.open();
- TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
- RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ RecordTupleGenerator tupleGenerator =
+ new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+ RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
VSizeFrame frame = new VSizeFrame(ctx);
VSizeFrame marker = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
@@ -178,9 +172,9 @@
nc.newJobId();
TestTupleCounterFrameWriter countOp = create(nc.getSearchOutputDesc(KEY_TYPES, RECORD_TYPE, META_TYPE),
Collections.emptyList(), Collections.emptyList(), false);
- IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, dataset, KEY_TYPES, RECORD_TYPE,
- META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES, KEY_INDICATORS_LIST,
- storageManager);
+ IPushRuntime emptyTupleOp = nc.getFullScanPipeline(countOp, ctx, StorageTestUtils.DATASET, KEY_TYPES,
+ RECORD_TYPE, META_TYPE, new NoMergePolicyFactory(), null, null, KEY_INDEXES,
+ KEY_INDICATORS_LIST, storageManager);
emptyTupleOp.open();
emptyTupleOp.close();
Assert.assertEquals(NUM_OF_RECORDS, countOp.getCount());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
index 62705cc..a7225a1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/MultiPartitionLSMIndexTest.java
@@ -29,8 +29,8 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
import org.apache.asterix.app.bootstrap.TestNodeController.SecondaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -302,7 +302,7 @@
}
@Override
- public void after() {
+ public void after(Void t) {
synchronized (allocated) {
allocated.setValue(true);
allocated.notifyAll();
@@ -339,7 +339,7 @@
}
@Override
- public void after() {
+ public void after(Semaphore t) {
}
});
synchronized (proceedToScheduleFlush) {
@@ -421,7 +421,7 @@
}
@Override
- public void after() {
+ public void after(Void t) {
synchronized (finishedSchduleFlush) {
finishedSchduleFlush.set(true);
finishedSchduleFlush.notifyAll();
@@ -478,7 +478,7 @@
}
@Override
- public void after() {
+ public void after(ILSMMemoryComponent t) {
synchronized (recycledPrimary) {
recycledPrimary.setValue(true);
recycledPrimary.notifyAll();
@@ -519,7 +519,7 @@
}
@Override
- public void after() {
+ public void after(ILSMMemoryComponent t) {
}
};
secondaryLsmBtrees[0].addIoRecycleCallback(secondaryRecycleCallback);
@@ -619,14 +619,14 @@
public class Actor extends SingleThreadEventProcessor<Request> {
private final int partition;
- private final TupleGenerator tupleGenerator;
+ private final RecordTupleGenerator tupleGenerator;
private final VSizeFrame frame;
private final FrameTupleAppender tupleAppender;
public Actor(String name, int partition) throws HyracksDataException {
super(name);
this.partition = partition;
- tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
+ tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
frame = new VSizeFrame(taskCtxs[partition]);
tupleAppender = new FrameTupleAppender(frame);
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
index e4373f6..61c1fb2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
@@ -26,8 +26,8 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -60,8 +60,9 @@
import org.apache.hyracks.storage.am.lsm.btree.impl.AllowTestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
import org.junit.After;
import org.junit.AfterClass;
@@ -160,8 +161,8 @@
// except search
lsmBtree.clearSearchCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
- RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+ KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
Searcher firstSearcher = null;
@@ -206,8 +207,8 @@
// except search
lsmBtree.clearSearchCallbacks();
insertOp.open();
- TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS,
- RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ RecordTupleGenerator tupleGenerator = new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES,
+ KEY_INDICATORS, RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
Searcher firstSearcher = null;
@@ -235,10 +236,11 @@
// merge all components
ILSMIndexAccessor mergeAccessor = lsmBtree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
List<ILSMDiskComponent> mergedComponents = new ArrayList<>(lsmBtree.getDiskComponents());
- BlockingIOOperationCallbackWrapper ioCallback =
- new BlockingIOOperationCallbackWrapper(lsmBtree.getIOOperationCallback());
- mergeAccessor.scheduleMerge(ioCallback, mergedComponents);
- ioCallback.waitForIO();
+ ILSMIOOperation merge = mergeAccessor.scheduleMerge(mergedComponents);
+ merge.sync();
+ if (merge.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(merge.getFailure());
+ }
// unblock the search
unblockSearch(lsmBtree);
// ensure the search got the correct number
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
index 390286a..589e8b2 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/StorageTestUtils.java
@@ -33,9 +33,9 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.app.data.gen.TestTupleCounterFrameWriter;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.context.DatasetInfo;
@@ -100,7 +100,7 @@
private StorageTestUtils() {
}
- static void allowAllOps(TestLsmBtree lsmBtree) {
+ public static void allowAllOps(TestLsmBtree lsmBtree) {
lsmBtree.clearModifyCallbacks();
lsmBtree.clearFlushCallbacks();
lsmBtree.clearSearchCallbacks();
@@ -118,6 +118,12 @@
KEY_INDICATORS_LIST, partition);
}
+ public static PrimaryIndexInfo createPrimaryIndex(TestNodeController nc, Dataset dataset, int partition)
+ throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+ return nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, STORAGE_MANAGER, KEY_INDEXES,
+ KEY_INDICATORS_LIST, partition);
+ }
+
public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx)
throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
return getInsertPipeline(nc, ctx, null);
@@ -131,13 +137,27 @@
}
public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+ Dataset dataset, Index secondaryIndex, IndexOperation op)
+ throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+ return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+ KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex, op).getLeft();
+ }
+
+ public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
Index secondaryIndex) throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
return nc.getInsertPipeline(ctx, DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
}
- public static TupleGenerator getTupleGenerator() {
- return new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
+ public static LSMInsertDeleteOperatorNodePushable getInsertPipeline(TestNodeController nc, IHyracksTaskContext ctx,
+ Dataset dataset, Index secondaryIndex)
+ throws HyracksDataException, RemoteException, ACIDException, AlgebricksException {
+ return nc.getInsertPipeline(ctx, dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, KEY_INDEXES,
+ KEY_INDICATORS_LIST, STORAGE_MANAGER, secondaryIndex).getLeft();
+ }
+
+ public static RecordTupleGenerator getTupleGenerator() {
+ return new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATORS, RECORD_GEN_FUNCTION,
UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
}
@@ -146,6 +166,11 @@
searchAndAssertCount(nc, partition, DATASET, STORAGE_MANAGER, numOfRecords);
}
+ public static void searchAndAssertCount(TestNodeController nc, Dataset dataset, int partition, int numOfRecords)
+ throws HyracksDataException, AlgebricksException {
+ searchAndAssertCount(nc, partition, dataset, STORAGE_MANAGER, numOfRecords);
+ }
+
public static void searchAndAssertCount(TestNodeController nc, int partition, Dataset dataset,
StorageComponentProvider storageManager, int numOfRecords)
throws HyracksDataException, AlgebricksException {
@@ -182,6 +207,11 @@
flushPartition(dslLifecycleMgr, lsmBtree, DATASET, async);
}
+ public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+ boolean async) throws Exception {
+ flushPartition(dslLifecycleMgr, lsmBtree, dataset, async);
+ }
+
public static void flushPartition(IDatasetLifecycleManager dslLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
boolean async) throws Exception {
waitForOperations(lsmBtree);
@@ -211,6 +241,11 @@
flush(dsLifecycleMgr, lsmBtree, DATASET, async);
}
+ public static void flush(IDatasetLifecycleManager dsLifecycleMgr, Dataset dataset, TestLsmBtree lsmBtree,
+ boolean async) throws Exception {
+ flush(dsLifecycleMgr, lsmBtree, dataset, async);
+ }
+
public static void flush(IDatasetLifecycleManager dsLifecycleMgr, TestLsmBtree lsmBtree, Dataset dataset,
boolean async) throws Exception {
waitForOperations(lsmBtree);
@@ -240,6 +275,11 @@
this(nc, partition, DATASET, STORAGE_MANAGER, lsmBtree, numOfRecords);
}
+ public Searcher(TestNodeController nc, Dataset dataset, int partition, TestLsmBtree lsmBtree,
+ int numOfRecords) {
+ this(nc, partition, dataset, STORAGE_MANAGER, lsmBtree, numOfRecords);
+ }
+
public Searcher(TestNodeController nc, int partition, Dataset dataset, StorageComponentProvider storageManager,
TestLsmBtree lsmBtree, int numOfRecords) {
lsmBtree.addSearchCallback(new ITestOpCallback<Semaphore>() {
@@ -253,7 +293,7 @@
}
@Override
- public void after() {
+ public void after(Semaphore t) {
}
});
Callable<Boolean> callable = new Callable<Boolean>() {
@@ -292,7 +332,7 @@
}
@Override
- public void after() {
+ public void after(Semaphore t) {
}
});
}
@@ -319,7 +359,7 @@
}
@Override
- public void after() {
+ public void after(Semaphore t) {
}
});
}
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 0c4983a..bcf68b5 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
@@ -21,6 +21,7 @@
import java.util.Map;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.metadata.entities.Dataset;
@@ -28,9 +29,14 @@
import org.apache.asterix.metadata.utils.DatasetUtil;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
+import org.apache.asterix.transaction.management.runtime.CommitRuntime;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntime;
+import org.apache.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.common.IResourceFactory;
@@ -48,6 +54,19 @@
}
@Override
+ public IPushRuntimeFactory getCommitRuntimeFactory(MetadataProvider metadataProvider,
+ int[] primaryKeyFieldPermutation, 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) };
+ }
+ };
+ }
+
+ @Override
public IResourceFactory getResourceFactory(MetadataProvider mdProvider, Index index, ARecordType recordType,
ARecordType metaType, ILSMMergePolicyFactory mergePolicyFactory, Map<String, String> mergePolicyProperties)
throws AlgebricksException {
@@ -62,6 +81,6 @@
@Override
public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
- return new TestLsmBtreeIoOpCallbackFactory(getComponentIdGeneratorFactory());
+ return new TestLsmIoOpCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
deleted file mode 100644
index 5852ad9..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmBtreeIoOpCallbackFactory.java
+++ /dev/null
@@ -1,170 +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.test.dataflow;
-
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
-
-public class TestLsmBtreeIoOpCallbackFactory extends LSMBTreeIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- private static volatile int completedFlushes = 0;
- private static volatile int completedMerges = 0;
- private static volatile int rollbackFlushes = 0;
- private static volatile int rollbackMerges = 0;
- private static volatile int failedFlushes = 0;
- private static volatile int failedMerges = 0;
-
- public TestLsmBtreeIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- super(idGeneratorFactory);
- }
-
- @Override
- public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
- completedFlushes = 0;
- completedMerges = 0;
- rollbackFlushes = 0;
- rollbackMerges = 0;
- // Whenever this is called, it resets the counter
- // However, the counters for the failed operations are never reset since we expect them
- // To be always 0
- return new TestLsmBtreeIoOpCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
- }
-
- public int getTotalFlushes() {
- return completedFlushes + rollbackFlushes;
- }
-
- public int getTotalMerges() {
- return completedMerges + rollbackMerges;
- }
-
- public int getTotalIoOps() {
- return getTotalFlushes() + getTotalMerges();
- }
-
- public int getRollbackFlushes() {
- return rollbackFlushes;
- }
-
- public int getRollbackMerges() {
- return rollbackMerges;
- }
-
- public int getCompletedFlushes() {
- return completedFlushes;
- }
-
- public int getCompletedMerges() {
- return completedMerges;
- }
-
- public static int getFailedFlushes() {
- return failedFlushes;
- }
-
- public static int getFailedMerges() {
- return failedMerges;
- }
-
- public class TestLsmBtreeIoOpCallback extends LSMBTreeIOOperationCallback {
- private final TestLsmBtree lsmBtree;
-
- public TestLsmBtreeIoOpCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider checkpointManagerProvider) {
- super(index, idGenerator, checkpointManagerProvider);
- lsmBtree = (TestLsmBtree) index;
- }
-
- @Override
- public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- lsmBtree.beforeIoOperationCalled();
- super.beforeOperation(opCtx);
- lsmBtree.beforeIoOperationReturned();
- }
-
- @Override
- public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- lsmBtree.afterIoOperationCalled();
- super.afterOperation(opCtx);
- lsmBtree.afterIoOperationReturned();
- }
-
- @Override
- public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- lsmBtree.afterIoFinalizeCalled();
- super.afterFinalize(opCtx);
- synchronized (TestLsmBtreeIoOpCallbackFactory.this) {
- if (opCtx.getNewComponent() != null) {
- if (opCtx.getNewComponent() == EmptyComponent.INSTANCE) {
- if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
- rollbackFlushes++;
- } else {
- rollbackMerges++;
- }
- } else {
- if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
- completedFlushes++;
- } else {
- completedMerges++;
- }
- }
- } else {
- recordFailure(opCtx.getIoOperationType());
- }
- TestLsmBtreeIoOpCallbackFactory.this.notifyAll();
- }
- lsmBtree.afterIoFinalizeReturned();
- }
-
- @Override
- public void recycled(ILSMMemoryComponent component, boolean advance) throws HyracksDataException {
- lsmBtree.recycledCalled(component);
- super.recycled(component, advance);
- lsmBtree.recycledReturned(component);
- }
-
- @Override
- public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
- lsmBtree.allocatedCalled(component);
- super.allocated(component);
- lsmBtree.allocatedReturned(component);
- }
-
- private void recordFailure(LSMIOOperationType opType) {
- if (opType == LSMIOOperationType.FLUSH) {
- failedFlushes++;
- } else {
- failedMerges++;
- }
- }
- }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
new file mode 100644
index 0000000..c762c8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestLsmIoOpCallbackFactory.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.dataflow;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.EmptyComponent;
+
+public class TestLsmIoOpCallbackFactory extends LSMIndexIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static volatile int completedFlushes = 0;
+ private static volatile int completedMerges = 0;
+ private static volatile int rollbackFlushes = 0;
+ private static volatile int rollbackMerges = 0;
+ private static volatile int failedFlushes = 0;
+ private static volatile int failedMerges = 0;
+
+ public TestLsmIoOpCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+ IDatasetInfoProvider datasetInfoProvider) {
+ super(idGeneratorFactory, datasetInfoProvider);
+ }
+
+ @Override
+ public synchronized ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+ completedFlushes = 0;
+ completedMerges = 0;
+ rollbackFlushes = 0;
+ rollbackMerges = 0;
+ // Whenever this is called, it resets the counter
+ // However, the counters for the failed operations are never reset since we expect them
+ // To be always 0
+ return new TestLsmIoOpCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+ getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+ }
+
+ public int getTotalFlushes() {
+ return completedFlushes + rollbackFlushes;
+ }
+
+ public int getTotalMerges() {
+ return completedMerges + rollbackMerges;
+ }
+
+ public int getTotalIoOps() {
+ return getTotalFlushes() + getTotalMerges();
+ }
+
+ public int getRollbackFlushes() {
+ return rollbackFlushes;
+ }
+
+ public int getRollbackMerges() {
+ return rollbackMerges;
+ }
+
+ public int getCompletedFlushes() {
+ return completedFlushes;
+ }
+
+ public int getCompletedMerges() {
+ return completedMerges;
+ }
+
+ public static int getFailedFlushes() {
+ return failedFlushes;
+ }
+
+ public static int getFailedMerges() {
+ return failedMerges;
+ }
+
+ public class TestLsmIoOpCallback extends LSMIOOperationCallback {
+ private final TestLsmBtree lsmBtree;
+
+ public TestLsmIoOpCallback(DatasetInfo dsInfo, ILSMIndex index, ILSMComponentId id,
+ IIndexCheckpointManagerProvider checkpointManagerProvider) {
+ super(dsInfo, index, id, checkpointManagerProvider);
+ lsmBtree = (TestLsmBtree) index;
+ }
+
+ @Override
+ public void scheduled(ILSMIOOperation op) throws HyracksDataException {
+ lsmBtree.ioScheduledCalled();
+ super.scheduled(op);
+ lsmBtree.ioScheduledReturned();
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation op) throws HyracksDataException {
+ lsmBtree.beforeIoOperationCalled();
+ super.beforeOperation(op);
+ lsmBtree.beforeIoOperationReturned();
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation op) throws HyracksDataException {
+ lsmBtree.afterIoOperationCalled();
+ super.afterOperation(op);
+ lsmBtree.afterIoOperationReturned();
+ }
+
+ @Override
+ public void afterFinalize(ILSMIOOperation op) throws HyracksDataException {
+ lsmBtree.afterIoFinalizeCalled();
+ super.afterFinalize(op);
+ synchronized (TestLsmIoOpCallbackFactory.this) {
+ if (op.getNewComponent() != null) {
+ if (op.getNewComponent() == EmptyComponent.INSTANCE) {
+ if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ rollbackFlushes++;
+ } else {
+ rollbackMerges++;
+ }
+ } else {
+ if (op.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ completedFlushes++;
+ } else {
+ completedMerges++;
+ }
+ }
+ } else {
+ recordFailure(op.getIOOpertionType());
+ }
+ TestLsmIoOpCallbackFactory.this.notifyAll();
+ }
+ lsmBtree.afterIoFinalizeReturned();
+ }
+
+ @Override
+ public void completed(ILSMIOOperation operation) {
+ try {
+ lsmBtree.ioCompletedCalled();
+ super.completed(operation);
+ lsmBtree.ioCompletedReturned();
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+ lsmBtree.recycledCalled(component);
+ super.recycled(component);
+ lsmBtree.recycledReturned(component);
+ }
+
+ @Override
+ public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+ lsmBtree.allocatedCalled(component);
+ super.allocated(component);
+ lsmBtree.allocatedReturned(component);
+ }
+
+ private void recordFailure(LSMIOOperationType opType) {
+ if (opType == LSMIOOperationType.FLUSH) {
+ failedFlushes++;
+ } else {
+ failedMerges++;
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
index 9a528d3..250c25f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPrimaryIndexOperationTracker.java
@@ -60,7 +60,7 @@
super.triggerScheduleFlush(logRecord);
synchronized (callbacks) {
for (ITestOpCallback<Void> callback : callbacks) {
- callback.after();
+ callback.after(null);
}
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
index 06b213d..bee2f8d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TransactionAbortTest.java
@@ -23,7 +23,7 @@
import org.apache.asterix.app.bootstrap.TestNodeController;
import org.apache.asterix.app.bootstrap.TestNodeController.PrimaryIndexInfo;
-import org.apache.asterix.app.data.gen.TupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
import org.apache.asterix.app.nc.NCAppRuntimeContext;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
@@ -68,7 +68,7 @@
private static IHyracksTaskContext abortCtx;
private static ITransactionContext abortTxnCtx;
private static LSMInsertDeleteOperatorNodePushable abortOp;
- private static TupleGenerator tupleGenerator;
+ private static RecordTupleGenerator tupleGenerator;
@Rule
public TestRule watcher = new TestMethodTracer();
@@ -199,7 +199,7 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
// manually set the current memory component as modified
index.getCurrentMemoryComponent().setModified();
throw new HyracksDataException("Fail the job");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
index 418282e..3634bf1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/logging/CheckpointingTest.java
@@ -18,6 +18,10 @@
*/
package org.apache.asterix.test.logging;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.spy;
+
import java.io.File;
import java.nio.file.Path;
import java.util.ArrayList;
@@ -26,10 +30,9 @@
import java.util.List;
import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator;
+import org.apache.asterix.app.data.gen.RecordTupleGenerator.GenerationFunction;
import org.apache.asterix.app.nc.RecoveryManager;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.Checkpoint;
@@ -43,14 +46,12 @@
import org.apache.asterix.common.utils.TransactionUtil;
import org.apache.asterix.external.util.DataflowUtils;
import org.apache.asterix.file.StorageComponentProvider;
-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.entities.InternalDatasetDetails.PartitioningStrategy;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.test.common.TestHelper;
+import org.apache.asterix.test.dataflow.StorageTestUtils;
import org.apache.asterix.transaction.management.service.logging.LogManager;
import org.apache.asterix.transaction.management.service.recovery.AbstractCheckpointManager;
import org.apache.asterix.transaction.management.service.transaction.TransactionManager;
@@ -60,17 +61,12 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.stubbing.Answer;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.spy;
-
public class CheckpointingTest {
private static final String TEST_CONFIG_FILE_NAME = "cc-small-txn-log-partition.conf";
@@ -116,23 +112,21 @@
nc.init();
List<List<String>> partitioningKeys = new ArrayList<>();
partitioningKeys.add(Collections.singletonList("key"));
- Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
- NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
- partitioningKeys, null, null, null, false, null),
- null, DatasetType.INTERNAL, DATASET_ID, 0);
try {
- nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
- KEY_INDICATOR_LIST, 0);
+ nc.createPrimaryIndex(StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager,
+ KEY_INDEXES, KEY_INDICATOR_LIST, 0);
JobId jobId = nc.newJobId();
IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
// Prepare insert operation
- LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
- RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+ LSMInsertDeleteOperatorNodePushable insertOp =
+ nc.getInsertPipeline(ctx, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
insertOp.open();
- TupleGenerator tupleGenerator = new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
- RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
+ RecordTupleGenerator tupleGenerator =
+ new RecordTupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR,
+ RECORD_GEN_FUNCTION, UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
VSizeFrame frame = new VSizeFrame(ctx);
FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
@@ -197,8 +191,9 @@
nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx2),
new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
// Prepare insert operation
- LSMInsertDeleteOperatorNodePushable insertOp2 = nc.getInsertPipeline(ctx2, dataset, KEY_TYPES,
- RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
+ LSMInsertDeleteOperatorNodePushable insertOp2 =
+ nc.getInsertPipeline(ctx2, StorageTestUtils.DATASET, KEY_TYPES, RECORD_TYPE, META_TYPE, null,
+ KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
insertOp2.open();
VSizeFrame frame2 = new VSizeFrame(ctx2);
FrameTupleAppender tupleAppender2 = new FrameTupleAppender(frame2);
@@ -220,6 +215,7 @@
}
}
Thread.UncaughtExceptionHandler h = new Thread.UncaughtExceptionHandler() {
+ @Override
public void uncaughtException(Thread th, Throwable ex) {
threadException = true;
exception = ex;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
index 0f6adf6..d5985b5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -51,7 +51,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.test.support.TestUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -261,8 +260,9 @@
final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxn);
final String nodeGroupName = "ng";
+ final String committedNodeGroup = "committed_ng";
+ final List<String> ngNodes = Collections.singletonList("asterix_nc1");
try {
- final List<String> ngNodes = Collections.singletonList("asterix_nc1");
MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup(nodeGroupName, ngNodes));
MetadataManager.INSTANCE.commitTransaction(mdTxn);
} finally {
@@ -281,6 +281,9 @@
int diskComponentsBeforeFlush = index.getDiskComponents().size();
// lock opTracker to prevent log flusher from triggering flush
synchronized (opTracker) {
+ final MetadataTransactionContext committedMdTxn = MetadataManager.INSTANCE.beginTransaction();
+ MetadataManager.INSTANCE.addNodegroup(committedMdTxn, new NodeGroup(committedNodeGroup, ngNodes));
+ MetadataManager.INSTANCE.commitTransaction(committedMdTxn);
opTracker.setFlushOnExit(true);
opTracker.flushIfNeeded();
Assert.assertTrue(opTracker.isFlushLogCreated());
@@ -288,7 +291,6 @@
// make sure force operation will processed
MetadataManager.INSTANCE.dropNodegroup(mdTxn2, nodeGroupName, false);
Assert.assertEquals(1, opTracker.getNumActiveOperations());
- Assert.assertFalse(index.hasFlushRequestForCurrentMutableComponent());
// release opTracker lock now to allow log flusher to schedule the flush
InvokeUtil.runWithTimeout(() -> {
synchronized (opTracker) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
deleted file mode 100644
index 09758f6..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
+++ /dev/null
@@ -1,72 +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.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with full parallelism on node controllers.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionFullParallelismIT {
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc2.conf";
-
- @BeforeClass
- public static void setUp() throws Exception {
- LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- LangExecutionUtil.tearDown();
- }
-
- @Parameters(name = "AqlExecutionFullParallelismIT {index}: {0}")
- public static Collection<Object[]> tests() throws Exception {
- Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
- if (!tests.isEmpty()) {
- tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
- } else {
- tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
- tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
- }
- return tests;
- }
-
- protected TestCaseContext tcCtx;
-
- public AqlExecutionFullParallelismIT(TestCaseContext tcCtx) {
- this.tcCtx = tcCtx;
- }
-
- @Test
- public void test() throws Exception {
- LangExecutionUtil.test(tcCtx);
- }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
deleted file mode 100644
index 62ed790..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
+++ /dev/null
@@ -1,72 +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.test.runtime;
-
-import java.util.Collection;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the AQL runtime tests with less parallelism on node controllers than using all the cores.
- */
-@RunWith(Parameterized.class)
-public class AqlExecutionLessParallelismIT {
- protected static final String TEST_CONFIG_FILE_NAME = "src/main/resources/cc3.conf";
-
- @BeforeClass
- public static void setUp() throws Exception {
- LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, new TestExecutor());
- }
-
- @AfterClass
- public static void tearDown() throws Exception {
- LangExecutionUtil.tearDown();
- }
-
- @Parameters(name = "AqlExecutionLessParallelismIT {index}: {0}")
- public static Collection<Object[]> tests() throws Exception {
- Collection<Object[]> tests = LangExecutionUtil.buildTestsInXml("only_it.xml");
- if (!tests.isEmpty()) {
- tests.addAll(LangExecutionUtil.buildTestsInXml("only.xml"));
- } else {
- tests = LangExecutionUtil.buildTestsInXml("testsuite_it.xml");
- tests.addAll(LangExecutionUtil.tests("only.xml", "testsuite.xml"));
- }
- return tests;
- }
-
- protected TestCaseContext tcCtx;
-
- public AqlExecutionLessParallelismIT(TestCaseContext tcCtx) {
- this.tcCtx = tcCtx;
- }
-
- @Test
- public void test() throws Exception {
- LangExecutionUtil.test(tcCtx);
- }
-}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index c7ae2df..62c882d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -25,7 +25,6 @@
import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.external.util.ExternalDataConstants;
@@ -107,6 +106,10 @@
tearDown(cleanup, integrationUtil, true);
}
+ public static void tearDown(boolean cleanup, boolean stopHdfs) throws Exception {
+ tearDown(cleanup, integrationUtil, stopHdfs);
+ }
+
public static void tearDown(boolean cleanup, AsterixHyracksIntegrationUtil integrationUtil, boolean stopHdfs)
throws Exception {
// validateBufferCacheState(); <-- Commented out until bug is fixed -->
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
deleted file mode 100644
index 17509a4..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/DiskIsFullTest.java
+++ /dev/null
@@ -1,182 +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.test.storage;
-
-import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
-
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
-import org.apache.asterix.app.bootstrap.TestNodeController;
-import org.apache.asterix.app.data.gen.TupleGenerator;
-import org.apache.asterix.app.data.gen.TupleGenerator.GenerationFunction;
-import org.apache.asterix.common.config.DatasetConfig.DatasetType;
-import org.apache.asterix.common.dataflow.LSMInsertDeleteOperatorNodePushable;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.TransactionOptions;
-import org.apache.asterix.external.util.DataflowUtils;
-import org.apache.asterix.file.StorageComponentProvider;
-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.entities.InternalDatasetDetails.PartitioningStrategy;
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.test.common.TestHelper;
-import org.apache.commons.lang3.SystemUtils;
-import org.apache.hyracks.api.comm.VSizeFrame;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicyFactory;
-import org.apache.hyracks.util.DiskUtil;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class DiskIsFullTest {
-
- private static final IAType[] KEY_TYPES = { BuiltinType.AINT32 };
- private static final ARecordType RECORD_TYPE = new ARecordType("TestRecordType", new String[] { "key", "value" },
- new IAType[] { BuiltinType.AINT32, BuiltinType.AINT64 }, false);
- private static final GenerationFunction[] RECORD_GEN_FUNCTION =
- { GenerationFunction.DETERMINISTIC, GenerationFunction.DETERMINISTIC };
- private static final boolean[] UNIQUE_RECORD_FIELDS = { true, false };
- private static final ARecordType META_TYPE = null;
- private static final GenerationFunction[] META_GEN_FUNCTION = null;
- private static final boolean[] UNIQUE_META_FIELDS = null;
- private static final int[] KEY_INDEXES = { 0 };
- private static final int[] KEY_INDICATOR = { Index.RECORD_INDICATOR };
- private static final List<Integer> KEY_INDICATOR_LIST = Arrays.asList(new Integer[] { Index.RECORD_INDICATOR });
- private static final int DATASET_ID = 101;
- private static final String DATAVERSE_NAME = "TestDV";
- private static final String DATASET_NAME = "TestDS";
- private static final String DATA_TYPE_NAME = "DUMMY";
- private static final String NODE_GROUP_NAME = "DEFAULT";
- private static final String TEST_DISK_NAME = "asterixdb_ram_disk";
- private boolean shouldRun = true;
-
- @Before
- public void setUp() throws Exception {
- if (!SystemUtils.IS_OS_MAC) {
- System.out.println("Skipping test " + DiskIsFullTest.class.getName() + " due to unsupported OS");
- shouldRun = false;
- return;
- }
- System.out.println("SetUp: ");
- TestHelper.deleteExistingInstanceFiles();
- // create RAM disk
- final Path ramDiskRoot = DiskUtil.mountRamDisk(TEST_DISK_NAME, 4, MEGABYTE);
- // Use RAM disk for storage
- AsterixHyracksIntegrationUtil.setStoragePath(ramDiskRoot.toAbsolutePath().toString());
- }
-
- @After
- public void tearDown() throws Exception {
- if (!shouldRun) {
- return;
- }
- System.out.println("TearDown");
- TestHelper.deleteExistingInstanceFiles();
- DiskUtil.unmountRamDisk(TEST_DISK_NAME);
- AsterixHyracksIntegrationUtil.restoreDefaultStoragePath();
- }
-
- @Test
- public void testDiskIsFull() {
- if (!shouldRun) {
- return;
- }
- HyracksDataException expectedException =
- HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
- try {
- TestNodeController nc = new TestNodeController(null, false);
- nc.init();
- StorageComponentProvider storageManager = new StorageComponentProvider();
- List<List<String>> partitioningKeys = new ArrayList<>();
- partitioningKeys.add(Collections.singletonList("key"));
- Dataset dataset = new Dataset(DATAVERSE_NAME, DATASET_NAME, DATAVERSE_NAME, DATA_TYPE_NAME, NODE_GROUP_NAME,
- NoMergePolicyFactory.NAME, null, new InternalDatasetDetails(null, PartitioningStrategy.HASH,
- partitioningKeys, null, null, null, false, null),
- null, DatasetType.INTERNAL, DATASET_ID, 0);
- try {
- nc.createPrimaryIndex(dataset, KEY_TYPES, RECORD_TYPE, META_TYPE, null, storageManager, KEY_INDEXES,
- KEY_INDICATOR_LIST, 0);
- JobId jobId = nc.newJobId();
- IHyracksTaskContext ctx = nc.createTestContext(jobId, 0, false);
- ITransactionContext txnCtx = nc.getTransactionManager().beginTransaction(nc.getTxnJobId(ctx),
- new TransactionOptions(ITransactionManager.AtomicityLevel.ENTITY_LEVEL));
- // Prepare insert operation
- LSMInsertDeleteOperatorNodePushable insertOp = nc.getInsertPipeline(ctx, dataset, KEY_TYPES,
- RECORD_TYPE, META_TYPE, null, KEY_INDEXES, KEY_INDICATOR_LIST, storageManager, null).getLeft();
- try {
- insertOp.open();
- TupleGenerator tupleGenerator =
- new TupleGenerator(RECORD_TYPE, META_TYPE, KEY_INDEXES, KEY_INDICATOR, RECORD_GEN_FUNCTION,
- UNIQUE_RECORD_FIELDS, META_GEN_FUNCTION, UNIQUE_META_FIELDS);
- VSizeFrame frame = new VSizeFrame(ctx);
- FrameTupleAppender tupleAppender = new FrameTupleAppender(frame);
- // Insert records until disk becomes full
- int tupleCount = 100000;
- while (tupleCount > 0) {
- ITupleReference tuple = tupleGenerator.next();
- try {
- DataflowUtils.addTupleToFrame(tupleAppender, tuple, insertOp);
- } catch (Throwable t) {
- final Throwable rootCause = ExceptionUtils.getRootCause(t);
- rootCause.printStackTrace();
- if (rootCause instanceof HyracksDataException) {
- HyracksDataException cause = (HyracksDataException) rootCause;
- Assert.assertEquals(cause.getErrorCode(), expectedException.getErrorCode());
- Assert.assertEquals(cause.getMessage(), expectedException.getMessage());
- return;
- } else {
- break;
- }
- }
- tupleCount--;
- }
- Assert.fail("Expected exception (" + expectedException + ") was not thrown");
- } finally {
- try {
- insertOp.close();
- } finally {
- nc.getTransactionManager().abortTransaction(txnCtx.getTxnId());
- }
- }
- } finally {
- nc.deInit();
- }
- } catch (Throwable e) {
- e.printStackTrace();
- Assert.fail("Expected exception (" + expectedException + ") was not thrown");
- }
- }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
index 1c8ac62..4453a1d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/txn/LogManagerTest.java
@@ -28,6 +28,7 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.config.TransactionProperties;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.ExceptionUtils;
@@ -64,6 +65,8 @@
@Before
public void setUp() throws Exception {
System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+ // use a small page size for test purpose
+ integrationUtil.addOption(TransactionProperties.Option.TXN_LOG_BUFFER_PAGESIZE, 128 * 1024);
integrationUtil.init(true, TEST_CONFIG_FILE_NAME);
}
diff --git a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
index 6cd3b87..811a40d 100644
--- a/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc-small-txn-log-partition.conf
@@ -52,5 +52,6 @@
messaging.frame.size=4096
messaging.frame.count=512
txn.log.partitionsize=2MB
+txn.log.buffer.pagesize=128KB
txn.log.checkpoint.pollfrequency=2147483647
storage.max.active.writable.datasets=50
\ No newline at end of file
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 822d2c3..5fc2c1e 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
@@ -50,6 +50,7 @@
{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
+{ "DataverseName": "Metadata", "DatatypeName": "geometry", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
index c0ab7a4..956cea1 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -339,73 +339,73 @@
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_3">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_4">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_5">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_6">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_7">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_239_drop_system_dataset_8">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
+ <expected-error>Invalid operation - Cannot drop a dataset belonging to the dataverse:Metadata</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_251_dataset_hint_error_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
+ <expected-error>Invalid operation - Dataset: Book error in processing hint: TUPLE_SIZE Unknown hint</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_251_dataset_hint_error_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
+ <expected-error>Invalid operation - Dataset: Book error in processing hint: SIZE Unknown hint</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_272_create_index_error_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+ <expected-error>There is no dataset with this name Foo in dataverse test</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="exception">
<compilation-unit name="issue_272_create_index_error_2">
<output-dir compare="Text">none</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException</expected-error>
+ <expected-error>There is no dataset with this name Foo in dataverse test</expected-error>
</compilation-unit>
</test-case>
<!-- This case should be fixed to return a proper message rather than NPE -->
@@ -444,37 +444,37 @@
<test-case FilePath="exception">
<compilation-unit name="issue_384_create_index_error_1">
<output-dir compare="Text">none</output-dir>
- <expected-error>Error: The field "[loc]" which is of type point cannot be indexed using the BTree index.</expected-error>
+ <expected-error>The field "[loc]" which is of type point cannot be indexed using the BTree index.</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>Error: The field "[age]" which is of type integer cannot be indexed using the RTree index.</expected-error>
+ <expected-error>The field "[age]" which is of type integer cannot be indexed using the RTree index.</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>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+ <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</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>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</expected-error>
+ <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned Keyword index.</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>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+ <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</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>Error: The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
+ <expected-error>The field "[loc]" which is of type point cannot be indexed using the Length Partitioned N-Gram index.</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -482,7 +482,7 @@
<test-case FilePath="transaction">
<compilation-unit name="failure_previous_success">
<output-dir compare="Text">failure_previous_success</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A datatype with this name StreetType already exists</expected-error>
+ <expected-error>A datatype with this name StreetType already exists</expected-error>
</compilation-unit>
<compilation-unit name="verify_failure_previous_success">
<output-dir compare="Text">verify_failure_previous_success</output-dir>
@@ -491,11 +491,11 @@
<test-case FilePath="transaction">
<compilation-unit name="failure_subsequent_no_execution">
<output-dir compare="Text">failure_subsequent_no_execution</output-dir>
- <expected-error>org.apache.hyracks.algebricks.common.exceptions.AlgebricksException: A nodegroup with this name group1 already exists</expected-error>
+ <expected-error>A nodegroup with this name group1 already exists</expected-error>
</compilation-unit>
<compilation-unit name="verify_failure_subsequent_no_execution">
<output-dir compare="Text">verify_failure_subsequent_no_execution</output-dir>
</compilation-unit>
</test-case>
</test-group>
-</test-suite>
+</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
deleted file mode 100644
index e757455..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.aql
+++ /dev/null
@@ -1,73 +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.
- */
-/*
- * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
- * Issue : 730, 741
- * Expected Res : Success
- * Date : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-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";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"count1":$t1.countA,
-"t2info": for $t2 in dataset('TweetMessages')
- where $t1.countA /* +indexnl */= $t2.countB and
- $t1.tweetid != $t2.tweetid
- order by $t2.tweetid
- return {"tweetid2": $t2.tweetid,
- "count2":$t2.countB}
-};
-
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
new file mode 100644
index 0000000..96e43a9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.sqlpp
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+ `screen-name` : string,
+ lang : string,
+ `friends-count` : integer,
+ `statuses-count` : integer,
+ name : string,
+ `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+ tweetid : bigint,
+ user : TwitterUserType,
+ `sender-location` : point,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string,
+ countA : integer,
+ countB : integer
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
+
+create index msgCountAIx on TweetMessages (countA) type btree;
+
+create index msgCountBIx on TweetMessages (countB) type btree;
+
+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
+ where ((t1.countA /*+ indexnl */ = t2.countB) and (t1.tweetid != t2.tweetid))
+ order by t2.tweetid
+ )}
+from TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
deleted file mode 100644
index 1775c6a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.aql
+++ /dev/null
@@ -1,64 +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.
- */
-/*
- * Description : Equi joins two datasets, Customers and Orders, based on the customer id.
- * Given the 'indexnl' hint we expect the join to be transformed
- * into an indexed nested-loop join using Customers' primary index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- lastorder: {
- oid: int32,
- total: float
- }
-}
-
-create type OrderType as closed {
- oid: int32,
- cid: int32,
- orderstatus: string,
- orderpriority: string,
- clerk: string,
- total: float
-}
-
-create dataset Customers(CustomerType) primary key cid;
-create dataset Orders(OrderType) primary key oid;
-
-write output to asterix_nc1:"rttest/btree-index-join_primary-equi-join_05.adm";
-
-for $o in dataset('Orders')
-for $c in dataset('Customers')
-where $o.cid /*+ indexnl */ = $c.cid
-return {"customer":$c, "order": $o}
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
new file mode 100644
index 0000000..e465dc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/primary-equi-join_04.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Equi joins two datasets, Customers and Orders, based on the customer id.
+ * Given the 'indexnl' hint we expect the join to be transformed
+ * into an indexed nested-loop join using Customers' primary index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ lastorder : {
+ oid : integer,
+ total : float
+ }
+
+};
+
+create type test.OrderType as
+ closed {
+ oid : integer,
+ cid : integer,
+ orderstatus : string,
+ orderpriority : string,
+ clerk : string,
+ total : float
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where (o.cid /*+ indexnl */ = c.cid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
deleted file mode 100644
index bb4ad1b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.aql
+++ /dev/null
@@ -1,78 +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.
- */
-/*
- * Description : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
- * We first expect FacebookUsers' primary index to be used
- * to satisfy the range condition on it's primary key.
- * FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint
- * we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type EmploymentType as closed {
- organization-name: string,
- start-date: date,
- end-date: date?
-}
-
-create type FacebookUserType as closed {
- id: int32,
- id-copy: int32,
- alias: string,
- name: string,
- user-since: datetime,
- user-since-copy: datetime,
- friend-ids: {{ int32 }},
- employment: [EmploymentType]
-}
-
-create type FacebookMessageType as closed {
- message-id: int32,
- message-id-copy: int32,
- author-id: int32,
- author-id-copy: int32,
- in-response-to: int32?,
- sender-location: point?,
- message: string
-}
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-create dataset FacebookMessages(FacebookMessageType)
-primary key message-id;
-
-create index fbmIdxAutId if not exists on FacebookMessages(author-id-copy);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join-multiindex.adm";
-
-for $user in dataset('FacebookUsers')
-for $message in dataset('FacebookMessages')
-where $user.id /*+ indexnl */ = $message.author-id-copy
-and $user.id >= 11000 and $user.id <= 12000
-return {
- "fbu-ID": $user.id,
- "fbm-auth-ID": $message.author-id,
- "uname": $user.name,
- "message": $message.message
-}
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
new file mode 100644
index 0000000..4580dab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join-multiindex.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Equi joins two datasets, FacebookUsers and FacebookMessages, based on their user's id.
+ * We first expect FacebookUsers' primary index to be used
+ * to satisfy the range condition on it's primary key.
+ * FacebookMessages has a secondary btree index on author-id-copy, and given the 'indexnl' hint
+ * we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.EmploymentType as
+ closed {
+ `organization-name` : string,
+ `start-date` : date,
+ `end-date` : date?
+};
+
+create type test.FacebookUserType as
+ closed {
+ id : integer,
+ `id-copy` : integer,
+ alias : string,
+ name : string,
+ `user-since` : datetime,
+ `user-since-copy` : datetime,
+ `friend-ids` : {{integer}},
+ employment : [EmploymentType]
+};
+
+create type test.FacebookMessageType as
+ closed {
+ `message-id` : integer,
+ `message-id-copy` : integer,
+ `author-id` : integer,
+ `author-id-copy` : integer,
+ `in-response-to` : integer?,
+ `sender-location` : point?,
+ message : string
+};
+
+create dataset FacebookUsers(FacebookUserType) primary key id;
+
+create dataset FacebookMessages(FacebookMessageType) primary key `message-id`;
+
+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
+where ((user.id /*+ indexnl */ = message.`author-id-copy`) and (user.id >= 11000) and (user.id <= 12000))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
deleted file mode 100644
index 5bdacc3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.aql
+++ /dev/null
@@ -1,57 +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.
- */
-/*
- * Description : Equi joins two datasets, DBLP and CSX, based on their title.
- * CSX has a secondary btree index on title, and given the 'indexnl' hint
- * we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index title_index on CSX(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $b}
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
new file mode 100644
index 0000000..1c315f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_02.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Equi joins two datasets, DBLP and CSX, based on their title.
+ * CSX has a secondary btree index on title, and given the 'indexnl' hint
+ * we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where (a.title /*+ indexnl */ = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
deleted file mode 100644
index 33bb320..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Equi self-joins a dataset, DBLP, based on its title.
- * DBLP has a secondary btree index on title, and given the 'indexnl' hint
- * we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index title_index on DBLP(title);
-
-write output to asterix_nc1:"rttest/btree-index-join_title-secondary-equi-join_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.title /*+ indexnl */ = $b.title
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..7ab67fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/secondary-equi-join_03.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 : Equi self-joins a dataset, DBLP, based on its title.
+ * DBLP has a secondary btree index on title, and given the 'indexnl' hint
+ * we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where (a.title /*+ indexnl */ = b.title)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
deleted file mode 100644
index b72dbcb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// Please note this is a Positive test and the BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Roger"
-return $emp
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
new file mode 100644
index 0000000..845da2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-33.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-33.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname > 'Roger')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
deleted file mode 100644
index 0030ddf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// This is a Positive test - prefix search, BTree index should be used in the plan.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Susan"
-return $emp
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
new file mode 100644
index 0000000..db55b28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-34.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-34.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname >= 'Susan')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
deleted file mode 100644
index 48b67d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification (usage) test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used.
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname < "Isa"
-return $emp
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
new file mode 100644
index 0000000..ecac451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-35.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 : BTree Index verification (usage) test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-35.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname < 'Isa')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
deleted file mode 100644
index 5e39552..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// Positive test - prefix search, BTree index should be used in query plan
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname <= "Vanpatten"
-return $emp
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
new file mode 100644
index 0000000..6354f25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-36.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-36.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname <= 'Vanpatten')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
deleted file mode 100644
index 23518e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used in the optimized query plan
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname = "Young Seok" and $emp.lname = "Kim"
-return $emp
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
new file mode 100644
index 0000000..d8a95cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-40.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.
+ */
+/*
+ * Description : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used in the optimized query plan
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-40.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname = 'Young Seok') and (emp.lname = 'Kim'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
deleted file mode 100644
index 81fb7b4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Alex" and $emp.lname < "Zach"
-return $emp
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
new file mode 100644
index 0000000..75ccb90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-42.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-42.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Alex') and (emp.lname < 'Zach'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
deleted file mode 100644
index fbb504d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Allan" and $emp.lname < "Zubi"
-return $emp
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
new file mode 100644
index 0000000..2cf7dbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-43.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-43.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname < 'Zubi'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
deleted file mode 100644
index fbc0dbb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Allan" and $emp.lname = "Xu"
-return $emp
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
new file mode 100644
index 0000000..98f037e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-44.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-44.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Allan') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
deleted file mode 100644
index 356503f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname = "Julio" and $emp.lname < "Xu"
-return $emp
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
new file mode 100644
index 0000000..a88838f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-45.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-45.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname < 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
deleted file mode 100644
index 9dcc4eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Michael" and $emp.lname <= "Xu"
-return $emp
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
new file mode 100644
index 0000000..524b0a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-46.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-46.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname >= 'Michael') and (emp.lname <= 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
deleted file mode 100644
index 5ffc387..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
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
new file mode 100644
index 0000000..f6a5120
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-47.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-47.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
deleted file mode 100644
index 4794082..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
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
new file mode 100644
index 0000000..cd4fae6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-48.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-48.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
deleted file mode 100644
index 47cc065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname <= "Craig" and $emp.lname > "Kevin"
-return $emp
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
new file mode 100644
index 0000000..9efcfc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-49.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-49.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname <= 'Craig') and (emp.lname > 'Kevin'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
deleted file mode 100644
index 9be2c17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname <= "Tomes"
-return $emp
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
new file mode 100644
index 0000000..3b3b5e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-51.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-51.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname <= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
deleted file mode 100644
index 807d861..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Craig" and $emp.lname >= "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
-return $emp
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
new file mode 100644
index 0000000..d6d977c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-52.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-52.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname >= 'Kevin') and (emp.fname < 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
deleted file mode 100644
index a7492fa..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Craig" and $emp.lname <= "Kevin" and $emp.fname <= "Mary" and $emp.lname >= "Tomes"
-return $emp
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
new file mode 100644
index 0000000..985cde7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-53.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-53.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname >= 'Craig') and (emp.lname <= 'Kevin') and (emp.fname <= 'Mary') and (emp.lname >= 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
deleted file mode 100644
index 7066b35..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Max"
-return $emp
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
new file mode 100644
index 0000000..40ae7d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-54.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-54.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname > 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
deleted file mode 100644
index 19168f0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Sofia"
-return $emp
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
new file mode 100644
index 0000000..8b9a588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-55.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-55.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname >= 'Sofia')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
deleted file mode 100644
index 09ad8c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname < "Chen"
-return $emp
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
new file mode 100644
index 0000000..14d1ce9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-56.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-56.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname < 'Chen')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
deleted file mode 100644
index 9c4d2ca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname <= "Julio"
-return $emp
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
new file mode 100644
index 0000000..4070114
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-57.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname <= 'Julio')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
deleted file mode 100644
index b49db5c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the primary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Neil" and $emp.fname < "Roger"
-return $emp
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
new file mode 100644
index 0000000..e3f42ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-58.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.
+ */
+/*
+ * Description : This test is intended to verify that the primary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-primary-58.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Neil') and (emp.fname < 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
deleted file mode 100644
index 479740f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname >= "Max" and $emp.fname <= "Roger"
-return $emp
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
new file mode 100644
index 0000000..8656ab8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-59.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
deleted file mode 100644
index 02ad02e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Description : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname = "Max"
-return $emp
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
new file mode 100644
index 0000000..38fad40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-60.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.
+ */
+/*
+ * Description : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-60.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where (emp.fname = 'Max')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
deleted file mode 100644
index 59801d3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 13th Aug 2012
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-// create internal dataset
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname < "Tomes"
-return $emp
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
new file mode 100644
index 0000000..e8d5627
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-61.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-61.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname > 'Craig') and (emp.lname > 'Kevin') and (emp.fname <= 'Mary') and (emp.lname < 'Tomes'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
deleted file mode 100644
index 90ad451..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname = "Julio" and $emp.lname > "Xu"
-return $emp
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
new file mode 100644
index 0000000..ce68496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-62.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 11th Nov 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-62.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname = 'Julio') and (emp.lname > 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
deleted file mode 100644
index 6b6bce0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : BTree Index verification test
- * : This test is intended to verify that the secondary BTree index is used
- * : in the optimized query plan.
- * Expected Result : Success
- * Date : 11th Nov 2014
- */
-
-// Positive test - prefix search
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname,lname);
-
-for $emp in dataset('testdst')
-where $emp.fname < "Julio" and $emp.lname = "Xu"
-return $emp
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
new file mode 100644
index 0000000..0ef6c12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-63.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 : BTree Index verification test
+ * : This test is intended to verify that the secondary BTree index is used
+ * : in the optimized query plan.
+ * Expected Result : Success
+ * Date : 11th Nov 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-63.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname,lname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname < 'Julio') and (emp.lname = 'Xu'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
deleted file mode 100644
index 8b8acaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
+++ /dev/null
@@ -1,49 +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.
- */
-drop dataverse colocated if exists;
-
-create dataverse colocated;
-
-use dataverse colocated;
-
-create type UserType as open {
- uid: int32,
- name: string,
- lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
- vid: int32,
- name: string,
- lottery_numbers: {{int32}}
-}
-
-create dataset Users(UserType)
- primary key uid;
-
-create dataset Visitors(VisitorType)
- primary key vid;
-
-
-write output to asterix_nc1:"/tmp/fuzzy1.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.uid = $visitor.vid
-return { "user_name" : $user.name, "visitor_name" : $visitor.name }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.sqlpp
new file mode 100644
index 0000000..ee49511
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/collocated.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.
+ */
+
+drop dataverse colocated if exists;
+create dataverse colocated;
+
+use colocated;
+
+
+create type colocated.UserType as
+{
+ uid : integer,
+ name : string,
+ lottery_numbers : {{integer}}
+};
+
+create type colocated.VisitorType as
+{
+ vid : integer,
+ name : string,
+ lottery_numbers : {{integer}}
+};
+
+create dataset Users(UserType) primary key uid;
+
+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
+where (user.uid = visitor.vid)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
deleted file mode 100644
index 2ad271e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.aql
+++ /dev/null
@@ -1,46 +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.
- */
-drop dataverse group_no_agg if exists;
-
-create dataverse group_no_agg;
-
-use dataverse group_no_agg;
-
-create type CustomerType as closed {
- c_custkey: int32,
- c_name: string,
- c_address: string,
- c_nationkey: int32,
- c_phone: string,
- c_acctbal: double,
- c_mktsegment: string,
- c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType)
- primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/.adm";
-
-for $c in dataset('Customers')
-group by $name := $c.c_name with $c
-return $name
-
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
new file mode 100644
index 0000000..3c82655
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/cust_group_no_agg.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.
+ */
+
+drop dataverse group_no_agg if exists;
+create dataverse group_no_agg;
+
+use group_no_agg;
+
+
+create type group_no_agg.CustomerType as
+ closed {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Customers(CustomerType) primary key c_custkey on 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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
deleted file mode 100644
index dbaf7be..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.aql
+++ /dev/null
@@ -1,68 +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.
- */
-drop dataverse custorder if exists;
-
-create dataverse custorder;
-
-use dataverse custorder;
-
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- lastorder: {
- oid: int32,
- total: float
- }
-}
-
-
-
-create type OrderType as closed {
- oid: int32,
- cid: int32,
- orderstatus: string,
- orderpriority: string,
- clerk: string,
- total: float
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Customers(CustomerType)
- primary key cid on group1;
-create dataset Orders(OrderType)
- primary key oid on group1;
-
-write output to asterix_nc1:"/tmp/custorder.adm";
-
-for $c in dataset('Customers')
-for $o in dataset('Orders')
-where $c.cid = $o.cid
-group by $cid := $c.cid decor $cust := $c with $o
-return {"cid":$cid, "cust":$cust, "cnt-orders": count($o), "orders":$o}
-
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
new file mode 100644
index 0000000..508d6ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/denorm-cust-order.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 custorder if exists;
+create dataverse custorder;
+
+use custorder;
+
+
+create type custorder.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type custorder.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ lastorder : {
+ oid : integer,
+ total : float
+ }
+
+};
+
+create type custorder.OrderType as
+ closed {
+ oid : integer,
+ cid : integer,
+ orderstatus : string,
+ orderpriority : string,
+ clerk : string,
+ total : float
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Customers(CustomerType) primary key cid on group1;
+
+create dataset Orders(OrderType) primary key oid on 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
+where (c.cid = o.cid)
+group by c.cid as cid group as g
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
deleted file mode 100644
index 285d27b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.aql
+++ /dev/null
@@ -1,32 +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.
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TestType as {
- "id" : string,
- "idx" : string,
- "no-idx" : string
-};
-
-create dataset TestSet(TestType) primary key "id";
-create index TestSetIndex on TestSet(idx);
-
-delete $x from dataset TestSet where $x.no-idx = "one" or $x.no-idx = "two";
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.sqlpp
new file mode 100644
index 0000000..40064ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/disjunction-to-join-delete-3.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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : string,
+ idx : string,
+ `no-idx` : string
+};
+
+create dataset TestSet(TestType) primary key id;
+
+create index TestSetIndex on TestSet (idx) type btree;
+
+delete from TestSet
+ where `no-idx` = 'one' or `no-idx` = 'two';
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
deleted file mode 100644
index 0425ecf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
+++ /dev/null
@@ -1,44 +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.
- */
-/*
- * Test case Name : insert-and-scan-dataset.aql
- * Description : This test is intended to test inserting into a dataset and scan it at the same time
- * where we insert a materializing to prevent the possibility of deadlatch.
- * Expected Result : Success
- * Date : July 11 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type myDataType as open {
- id: int32
-}
-
-create dataset myData(myDataType)
- primary key id;
-
-insert into dataset myData (
-for $x in dataset myData
-return {
- "id": $x.id + 1
-}
-);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.sqlpp
new file mode 100644
index 0000000..5566eaf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.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.
+ */
+/*
+ * Test case Name : insert-and-scan-dataset.aql
+ * Description : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date : July 11 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.myDataType as
+{
+ id : integer
+};
+
+create dataset myData(myDataType) primary key id;
+
+insert into myData
+select element {'id':(x.id + 1)}
+from myData as x
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
deleted file mode 100644
index 261a357..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.aql
+++ /dev/null
@@ -1,41 +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.
- */
-drop dataverse fuzzyjoin if exists;
-create dataverse fuzzyjoin;
-use dataverse fuzzyjoin;
-
-create type TOKENSRANKEDADMType as closed {
- token: int32,
- rank: int32
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
-
-write output to asterix_nc1:'rttest/introhashpartitionmerge.adm';
-
- for $token1 in dataset('TOKENSRANKEDADM')
- for $token2 in
- for $tokenRanked in dataset('TOKENSRANKEDADM')
- order by $tokenRanked.rank
- return $tokenRanked
- where $token1.token = $token2.token
- order by $token2.rank
- return $token2.rank
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
new file mode 100644
index 0000000..103aae8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.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.
+ */
+
+drop dataverse fuzzyjoin if exists;
+create dataverse fuzzyjoin;
+
+use fuzzyjoin;
+
+
+create type fuzzyjoin.TOKENSRANKEDADMType as
+ closed {
+ token : integer,
+ rank : integer
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
+
+write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
+select element token2.rank
+from TOKENSRANKEDADM as token1,
+ (
+ select element tokenRanked
+ from TOKENSRANKEDADM as tokenRanked
+ order by tokenRanked.rank
+) as token2
+where (token1.token = token2.token)
+order by token2.rank
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
deleted file mode 100644
index 488ed70..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- * The index should *not* be applied (see below).
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
-
-// Cannot optimize this query because the string constant is shorter than the gram length.
-for $o in dataset('DBLP')
-where contains($o.title, "Mu")
-order by $o.id
-return $o
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
new file mode 100644
index 0000000..e13305d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic.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 : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ * The index should *not* be applied (see below).
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
deleted file mode 100644
index ba460a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
-
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $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
new file mode 100644
index 0000000..e5b4885
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains.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 : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
deleted file mode 100644
index 81780a2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance-check.adm";
-
-for $o in dataset('DBLP')
-where edit-distance-check($o.authors, "Amihay Motro", 1)[0]
-return $o
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
new file mode 100644
index 0000000..da55c99
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance-check.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 : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
deleted file mode 100644
index 6917878..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.aql
+++ /dev/null
@@ -1,45 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-edit-distance.adm";
-
-for $o in dataset('DBLP')
-where edit-distance($o.authors, "Amihay Motro") <= 1
-return $o
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
new file mode 100644
index 0000000..91a809b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-edit-distance.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 : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance function on strings.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index b4d342c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $o in dataset('DBLP')
-where $o.authors ~= "Amihay Motro"
-return $o
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
new file mode 100644
index 0000000..d684f0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-edit-distance.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query with ~= using edit-distance on strings.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element o
+from DBLP as o
+where (o.authors ~= 'Amihay Motro')
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
deleted file mode 100644
index 55f1830..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $o in dataset('DBLP')
-where gram-tokens($o.title, 3, false) ~= gram-tokens("Transactions for Cooperative Environments", 3, false)
-return $o
diff --git a/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
new file mode 100644
index 0000000..5adcda0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.sqlpp
@@ -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.
+ */
+/*
+ * Description : Tests whether an ngram_index is applied to optimize a selection query with ~= using Jaccard on 3-gram tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element o
+from DBLP as o
+where (test.`gram-tokens`(o.title,3,false) ~= test.`gram-tokens`('Transactions for Cooperative Environments',3,false))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
deleted file mode 100644
index c7d273d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)[0]
-return $o
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
new file mode 100644
index 0000000..5b564db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.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 : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
deleted file mode 100644
index da87bc0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ngram-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false)) >= 0.5f
-return $o
diff --git a/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
new file mode 100644
index 0000000..3e024a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.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 : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard function on 3-gram tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
deleted file mode 100644
index 88b0d58..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- * The index should *not* be applied (see below).
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)[0]
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..41ceade
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ * The index should *not* be applied (see below).
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
deleted file mode 100644
index d107b68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
-
-for $c in dataset('Customers')
-where edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)[0]
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..0ca8b00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
deleted file mode 100644
index f4d5e14..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- * The index should *not* be applied (see below).
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
-
-// Index should not be applied because all list elements can be modified by 3 edit operations.
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 3
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..0f2fcf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ * The index should *not* be applied (see below).
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
deleted file mode 100644
index 687e55d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
-
-for $c in dataset('Customers')
-where edit-distance($c.interests, ["computers", "wine", "walking"]) <= 1
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..a4114d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
deleted file mode 100644
index 8833eaf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["computers", "wine", "walking"]
-order by $c.cid
-return $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
new file mode 100644
index 0000000..1bfd5f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance.sqlpp
@@ -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.
+ */
+/*
+ * Description : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element c
+from Customers as c
+where (c.interests ~= ['computers','wine','walking'])
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
deleted file mode 100644
index d0df75f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $c in dataset('Customers')
-where $c.interests ~= ["databases", "computers", "wine"]
-return $c
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
new file mode 100644
index 0000000..a1372d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-jaccard.sqlpp
@@ -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.
+ */
+/*
+ * Description : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element c
+from Customers as c
+where (c.interests ~= ['databases','computers','wine'])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
deleted file mode 100644
index 97e9962..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard-check.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)[0]
-return $c
diff --git a/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
new file mode 100644
index 0000000..89a3eb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
deleted file mode 100644
index 329b26a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.aql
+++ /dev/null
@@ -1,52 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_olist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, ["databases", "computers", "wine"]) >= 0.7f
-return $c
diff --git a/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
new file mode 100644
index 0000000..d13c98b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on lists.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
deleted file mode 100644
index c20493a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $c in dataset('Customers')
-where $c.interests ~= {{"computers", "wine", "databases"}}
-return $c
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
new file mode 100644
index 0000000..d32a8f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-fuzzyeq-jaccard.sqlpp
@@ -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.
+ */
+/*
+ * Description : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on sets.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element c
+from Customers as c
+where (c.interests ~= {{'computers','wine','databases'}})
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
deleted file mode 100644
index cb6c0c4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.aql
+++ /dev/null
@@ -1,52 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard-check($c.interests, {{"computers", "wine", "databases"}}, 0.7f)[0]
-return $c
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
new file mode 100644
index 0000000..25568e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard-check.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on sets.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
deleted file mode 100644
index c0cdd4f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.aql
+++ /dev/null
@@ -1,52 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_ulist-jaccard.adm";
-
-for $c in dataset('Customers')
-where similarity-jaccard($c.interests, {{"computers", "databases", "wine"}}) >= 0.7f
-return $c
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
new file mode 100644
index 0000000..2a26ee9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ulist-jaccard.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on sets.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
deleted file mode 100644
index 191a9bf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the contains function.
- * The index should *not* be applied (see below).
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
-
-// Contains cannot be answered with a word inverted index.
-for $o in dataset('DBLP')
-where contains($o.title, "Multimedia")
-order by $o.id
-return $o
diff --git a/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
new file mode 100644
index 0000000..2a1db6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains.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 : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ * The index should *not* be applied (see below).
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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')
+order by o.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
deleted file mode 100644
index ad2c577..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $o in dataset('DBLP')
-where word-tokens($o.title) ~= word-tokens("Transactions for Cooperative Environments")
-return $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
new file mode 100644
index 0000000..6157fde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-fuzzyeq-jaccard.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword is applied to optimize a selection query with ~= using Jaccard on word tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element o
+from DBLP as o
+where (test.`word-tokens`(o.title) ~= test.`word-tokens`('Transactions for Cooperative Environments'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
deleted file mode 100644
index 6f4497f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard-check.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)[0]
-return $o
-
diff --git a/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
new file mode 100644
index 0000000..b0dc05a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard-check.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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
deleted file mode 100644
index c96de32..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-basic_word-jaccard.adm";
-
-for $o in dataset('DBLP')
-where similarity-jaccard(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments")) >= 0.5f
-return $o
-
diff --git a/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
new file mode 100644
index 0000000..eb96ddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-jaccard.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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard function on word tokens.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
deleted file mode 100644
index cbe5b17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using
- * two edit-distance-check function of which only the first can be optimized with an index.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-// Only the first edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-where $eda[0] and $edb[0]
-return $o
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
new file mode 100644
index 0000000..c60c8fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query using
+ * two edit-distance-check function of which only the first can be optimized with an index.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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),
+ edb as test.`edit-distance-check`(o.authors,'Amihay Motro',5)
+where (eda[0] and edb[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
deleted file mode 100644
index a40a4de..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using
- * two edit-distance-check function of which only the second can be optimized with an index.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-// Only the second edit-distance-check can be optimized with an index.
-for $o in dataset('DBLP')
-let $edb := edit-distance-check($o.authors, "Amihay Motro", 5)
-let $eda := edit-distance-check($o.authors, "Amihay Motro", 3)
-where $edb[0] and $eda[0]
-return $o
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
new file mode 100644
index 0000000..8a5d1d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query using
+ * two edit-distance-check function of which only the second can be optimized with an index.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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),
+ eda as test.`edit-distance-check`(o.authors,'Amihay Motro',3)
+where (edb[0] and eda[0])
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
deleted file mode 100644
index 09173ce..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-for $o in dataset('DBLP')
-let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
-where $ed[0]
-return $o
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
new file mode 100644
index 0000000..1f725d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-let.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 : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on strings.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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)
+where ed[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
deleted file mode 100644
index 73293bc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
- * Tests that the optimizer rule correctly drills through the substring function.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-for $paper in dataset('DBLP')
-where edit-distance-check(substring($paper.title, 0, 8), "datbase", 1)[0]
-return {
- "id" : $paper.id,
- "title" : $paper.title
-}
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
new file mode 100644
index 0000000..d2e3888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-edit-distance-check-substring.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 : Tests whether an ngram_index index is applied to optimize a selection query using the similarity-edit-distance-check function on the substring of the field.
+ * Tests that the optimizer rule correctly drills through the substring function.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
deleted file mode 100644
index 3271c8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ngram-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
-where $jacc[0]
-return $o
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
new file mode 100644
index 0000000..248c243
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
deleted file mode 100644
index dfeda68..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := gram-tokens($paper.title, 3, false)
-let $query_tokens := gram-tokens("Transactions for Cooperative Environments", 3, false)
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
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
new file mode 100644
index 0000000..8be4caa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.sqlpp
@@ -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.
+ */
+/*
+ * Description : Tests whether an ngram_index is applied to optimize a selection query using the similarity-jaccard-check function on 3-gram tokens.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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),
+ query_tokens as test.`gram-tokens`('Transactions for Cooperative Environments',3,false),
+ jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.500000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
deleted file mode 100644
index 2cb65d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should *not* be applied (see below).
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 3)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..68106a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should *not* be applied (see below).
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
deleted file mode 100644
index 9646383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
-
-for $c in dataset('Customers')
-let $ed := edit-distance-check($c.interests, ["computers", "wine", "walking"], 1)
-where $ed[0]
-order by $c.cid
-return $c
diff --git a/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
new file mode 100644
index 0000000..7083657
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+where ed[0]
+order by c.cid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
deleted file mode 100644
index a274b9f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_olist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/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
new file mode 100644
index 0000000..a04a8ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
deleted file mode 100644
index 9d3ca6c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.aql
+++ /dev/null
@@ -1,54 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_ulist-jaccard-check-let.adm";
-
-for $c in dataset('Customers')
-let $jacc := similarity-jaccard-check($c.interests, ["databases", "computers", "wine"], 0.7f)
-where $jacc[0]
-return $c
diff --git a/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
new file mode 100644
index 0000000..ceac8c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ulist-jaccard-check-let.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on lists.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
deleted file mode 100644
index d932d23..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-let.adm";
-
-for $o in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
-where $jacc[0]
-return $o
-
diff --git a/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
new file mode 100644
index 0000000..e4f3b8d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-let.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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
deleted file mode 100644
index 79fe390..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
- * Tests that the optimizer rule correctly drills through the let clauses.
- * The index should be applied.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-complex_word-jaccard-check-multi-let.adm";
-
-// This test is complex because we have three assigns to drill into.
-for $paper in dataset('DBLP')
-let $paper_tokens := word-tokens($paper.title)
-let $query_tokens := word-tokens("Transactions for Cooperative Environments")
-let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.8f)
-where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
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
new file mode 100644
index 0000000..56ede6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/word-jaccard-check-multi-let.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Tests whether a keyword index is applied to optimize a selection query using the similarity-jaccard-check function on word tokens.
+ * Tests that the optimizer rule correctly drills through the let clauses.
+ * The index should be applied.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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),
+ query_tokens as test.`word-tokens`('Transactions for Cooperative Environments'),
+ jacc as test.`similarity-jaccard-check`(paper_tokens,query_tokens,0.800000f)
+where jacc[0]
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
deleted file mode 100644
index b99bc22..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors, "ed": $ed}
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
new file mode 100644
index 0000000..c29de43
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-edit-distance-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
deleted file mode 100644
index d4491eb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.aql
+++ /dev/null
@@ -1,60 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"aauthors": $a.authors, "bauthors": $b.authors}
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
new file mode 100644
index 0000000..1234824
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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`;
+
+select element {'aauthors':a.authors,'bauthors':b.authors}
+from DBLP as a,
+ CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
deleted file mode 100644
index 04d57a4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
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
new file mode 100644
index 0000000..4fdc637
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
deleted file mode 100644
index 5d48e10..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-edit-distance-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "ed": $ed}
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
new file mode 100644
index 0000000..40c8c5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
deleted file mode 100644
index d074b0f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_olist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc }
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
new file mode 100644
index 0000000..a003dee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/olist-jaccard-inline.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
deleted file mode 100644
index 56697c5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ulist-jaccard-inline.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"ainterests": $a.interests, "binterests": $b.interests, "jacc": $jacc}
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
new file mode 100644
index 0000000..1372490
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
deleted file mode 100644
index b917cda..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
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
new file mode 100644
index 0000000..9530163
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/word-jaccard-inline.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
deleted file mode 100644
index 8ddd03b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.aql
+++ /dev/null
@@ -1,65 +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.
- */
-/*
- * Description : Test that left-outer-join may use an available inverted index in index subtree.
- * Issue : 741
- * Expected Res : Success
- * Date : 16th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as {
-screen_name: string,
-lang: string,
-friends_count: int32,
-statuses_count: int32,
-name: string,
-followers_count: int32
-}
-
-create type TweetMessageType as {
-tweetid: int64,
-user: TwitterUserType,
-sender_location: point?,
-send_time: datetime,
-referred_topics: {{ string }},
-message_text: string
-}
-
-create dataset TweetMessages(TweetMessageType) primary key tweetid;
-
-create index topicIIx on TweetMessages(referred_topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_issue741.adm";
-
-for $t in dataset('TweetMessages')
-where $t.send_time >= datetime('2011-06-18T14:10:17')
-and
-$t.send_time < datetime('2011-06-18T15:10:17')
-return {
- "tweet": $t.tweetid,
- "similar-tweets": for $t2 in dataset('TweetMessages')
- let $sim := similarity-jaccard-check($t.referred_topics, $t2.referred_topics, 0.6f)
- where $sim[0] and
- $t2.tweetid != $t.tweetid
- return $t2.tweetid
-}
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
new file mode 100644
index 0000000..5a312b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/issue741.sqlpp
@@ -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.
+ */
+/*
+ * Description : Test that left-outer-join may use an available inverted index in index subtree.
+ * Issue : 741
+ * Expected Res : Success
+ * Date : 16th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+{
+ screen_name : string,
+ lang : string,
+ friends_count : integer,
+ statuses_count : integer,
+ name : string,
+ followers_count : integer
+};
+
+create type test.TweetMessageType as
+{
+ tweetid : bigint,
+ user : TwitterUserType,
+ sender_location : point?,
+ send_time : datetime,
+ referred_topics : {{string}},
+ message_text : string
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+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
+ with sim as test.`similarity-jaccard-check`(t.referred_topics,t2.referred_topics,0.600000f)
+ where (sim[0] and (t2.tweetid != t.tweetid))
+ )}
+from TweetMessages as t
+where ((t.send_time >= test.datetime('2011-06-18T14:10:17')) and (t.send_time < test.datetime('2011-06-18T15:10:17')))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
deleted file mode 100644
index bd52bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.aql
+++ /dev/null
@@ -1,73 +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.
- */
-/*
- * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue : 730, 741
- * Expected Res : Success
- * Date : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-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";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
- "tweet": {"id": $t1.tweetid, "topics" : $t1.message-text} ,
- "similar-tweets": for $t2 in dataset('TweetMessages')
- let $sim := edit-distance-check($t1.message-text, $t2.message-text, 7)
- where $sim[0] and
- $t2.tweetid != $t1.tweetid
- order by $t2.tweetid
- return {"id": $t2.tweetid, "topics" : $t2.message-text}
-};
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
new file mode 100644
index 0000000..8c2869f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+ `screen-name` : string,
+ lang : string,
+ `friends-count` : integer,
+ `statuses-count` : integer,
+ name : string,
+ `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+ tweetid : bigint,
+ user : TwitterUserType,
+ `sender-location` : point,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string,
+ countA : integer,
+ countB : integer
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
+
+create index msgCountAIx on TweetMessages (countA) type btree;
+
+create index msgCountBIx on TweetMessages (countB) type btree;
+
+create index msgKeywordIx on TweetMessages (`message-text`) type keyword;
+
+create index msgNgramIx on TweetMessages (`message-text`) type ngram (3);
+
+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
+ with sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+ where (sim[0] and (t2.tweetid != t1.tweetid))
+ order by t2.tweetid
+ )}
+from TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
deleted file mode 100644
index 27b0e37..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
+++ /dev/null
@@ -1,73 +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.
- */
-/*
- * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue : 730, 741
- * Expected Res : Success
- * Date : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgKeywordIx on TweetMessages(message-text) type keyword;
-create index msgNgramIx on TweetMessages(message-text) type ngram(3);
-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";
-
-for $t1 in dataset('TweetMessages')
-where $t1.tweetid > int64("240")
-order by $t1.tweetid
-return {
- "tweet": {"id": $t1.tweetid, "topics" : $t1.referred-topics} ,
- "similar-tweets": for $t2 in dataset('TweetMessages')
- let $sim := similarity-jaccard-check($t1.referred-topics, $t2.referred-topics, 0.5f)
- where $sim[0] and
- $t2.tweetid != $t1.tweetid
- order by $t2.tweetid
- return {"id": $t2.tweetid, "topics" : $t2.referred-topics}
-};
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
new file mode 100644
index 0000000..9db0bd7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.sqlpp
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+ `screen-name` : string,
+ lang : string,
+ `friends-count` : integer,
+ `statuses-count` : integer,
+ name : string,
+ `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+ tweetid : bigint,
+ user : TwitterUserType,
+ `sender-location` : point,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string,
+ countA : integer,
+ countB : integer
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
+
+create index msgCountAIx on TweetMessages (countA) type btree;
+
+create index msgCountBIx on TweetMessages (countB) type btree;
+
+create index msgKeywordIx on TweetMessages (`message-text`) type keyword;
+
+create index msgNgramIx on TweetMessages (`message-text`) type ngram (3);
+
+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
+ with sim as test.`similarity-jaccard-check`(t1.`referred-topics`,t2.`referred-topics`,0.500000f)
+ where (sim[0] and (t2.tweetid != t1.tweetid))
+ order by t2.tweetid
+ )}
+from TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
deleted file mode 100644
index cf2222d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
- * CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-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";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..a6ade32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance-check function of their authors.
+ * CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
deleted file mode 100644
index 95b2912..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance-check($a.authors, $b.authors, 3)[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..ed8d6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_03.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 : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where (test.`edit-distance-check`(a.authors,b.authors,3)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
deleted file mode 100644
index 5961d13..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance-check($a.authors, $b.authors, 3)
-where $ed[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
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
new file mode 100644
index 0000000..7a24772
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance-check_04.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance-check function of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with ed as test.`edit-distance-check`(a.authors,b.authors,3)
+where (ed[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
deleted file mode 100644
index c1a1817..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
- * CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..950a058
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based on the edit-distance function of their authors.
+ * CSX has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
deleted file mode 100644
index c65b61b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.aql
+++ /dev/null
@@ -1,46 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where edit-distance($a.authors, $b.authors) < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..dc8ca3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_03.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 : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where ((test.`edit-distance`(a.authors,b.authors) < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
deleted file mode 100644
index 6287cb0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(authors) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-edit-distance_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $ed := edit-distance($a.authors, $b.authors)
-where $ed < 3 and $a.id < $b.id
-return {"arec": $a, "brec": $b, "ed": $ed}
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
new file mode 100644
index 0000000..0284ff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-edit-distance_04.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the edit-distance function of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with ed as test.`edit-distance`(a.authors,b.authors)
+where ((ed < 3) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
deleted file mode 100644
index 66bb2ac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.aql
+++ /dev/null
@@ -1,59 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..2021605
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_01.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using edit distance of their authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ CSX as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c611d3a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
- * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where $a.authors ~= $b.authors and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..d900a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-edit-distance_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on ~= using edit distance of its authors.
+ * DBLP has a 3-gram index on authors, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ DBLP as b
+where ((a.authors ~= b.authors) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 085a383..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,60 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
- * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..aac9ca7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard their titles' 3-gram tokens.
+ * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ CSX as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 444b735..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..d3bcd49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ DBLP as b
+where ((test.`gram-tokens`(a.title,3,false) ~= test.`gram-tokens`(b.title,3,false)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
deleted file mode 100644
index 31ec7a5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
- * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..460f1cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
+ * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
deleted file mode 100644
index 6255558..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..5e180f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where (test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
deleted file mode 100644
index f70df08..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
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
new file mode 100644
index 0000000..2b9aa31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard-check`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
deleted file mode 100644
index 2e1b065..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
- * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..a2dc8f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.sqlpp
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
+ * CSX has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
deleted file mode 100644
index 31003d4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..fbc230c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where ((test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
deleted file mode 100644
index 1e86763..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
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
new file mode 100644
index 0000000..5265874
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
+ * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+set `import-private-functions` `true`;
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard`(test.`gram-tokens`(a.title,3,false),test.`gram-tokens`(b.title,3,false))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
deleted file mode 100644
index f074ad1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..f7e6047
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance-check function of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
deleted file mode 100644
index 723d0b5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance-check($a.interests, $b.interests, 3)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..6435990
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where (test.`edit-distance-check`(a.interests,b.interests,3)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
deleted file mode 100644
index 0bb95d5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance-check($a.interests, $b.interests, 3)
-where $ed[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed[1] }
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
new file mode 100644
index 0000000..9f14368
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the edit-distance-check function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with ed as test.`edit-distance-check`(a.interests,b.interests,3)
+where (ed[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
deleted file mode 100644
index e5f5b40..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..5ebb518
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the edit-distance function of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
deleted file mode 100644
index f78a6bd..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where edit-distance($a.interests, $b.interests) <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..ac7e6ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where ((test.`edit-distance`(a.interests,b.interests) <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
deleted file mode 100644
index c4e037f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-edit-distance_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $ed := edit-distance($a.interests, $b.interests)
-where $ed <= 2 and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "ed": $ed }
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
new file mode 100644
index 0000000..0655746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-edit-distance_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the edit-distance function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with ed as test.`edit-distance`(a.interests,b.interests)
+where ((ed <= 2) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
deleted file mode 100644
index c5a558c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..cd3d74f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on ~= using edit distance of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers2 as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
deleted file mode 100644
index c96c0e4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..a184602
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-edit-distance_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on ~= using edit distance of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers as b
+where ((a.interests ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index f90593a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..c1f3af7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers2 as b
+where ((a.interests /*+ indexnl */ ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 264c93a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..573f5e2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-fuzzyeq-jaccard_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers as b
+where ((a.interests /*+ indexnl */ ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
deleted file mode 100644
index ce60fe5..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..c9cbd1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
deleted file mode 100644
index edaba17..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..6364474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
deleted file mode 100644
index 3d704f2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
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
new file mode 100644
index 0000000..bd84396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
deleted file mode 100644
index 55c34ff..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..b6bea6a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest lists.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
deleted file mode 100644
index 37725d9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..3359746
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
deleted file mode 100644
index 1bba03c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: [string],
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_olist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
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
new file mode 100644
index 0000000..941d506
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/olist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest lists.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : [string],
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index b918bac..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..5c46ddc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_02.sqlpp
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on ~= using Jaccard of their interest sets.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers2 as b
+where ((a.interests /*+ indexnl */ ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 64260d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,56 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-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';
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where $a.interests /*+ indexnl */ ~= $b.interests and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..b874be2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-fuzzyeq-jaccard_03.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, Customers, based on ~= using Jaccard of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from Customers as a,
+ Customers as b
+where ((a.interests /*+ indexnl */ ~= b.interests) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
deleted file mode 100644
index 0a1f2f8..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b }
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
new file mode 100644
index 0000000..2a771e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard-check function of their interest sets.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
deleted file mode 100644
index f489f34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..8d71bb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where ( /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
deleted file mode 100644
index 6c87995..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard-check_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard-check($a.interests, $b.interests, 0.7f)
-where $jacc[0] and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
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
new file mode 100644
index 0000000..476db19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard-check_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard-check function of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard-check`(a.interests,b.interests,0.700000f)
+where (jacc[0] and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
deleted file mode 100644
index 8a6d743..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
- * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create dataset Customers2(CustomerType) primary key cid;
-
-create index interests_index on Customers2(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_02.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers2')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..287ef06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_02.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, Customer and Customer2, based on the similarity-jaccard function of their interest sets.
+ * Customers2 has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+create dataset Customers2(CustomerType) primary key cid;
+
+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
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
deleted file mode 100644
index ef5269a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.aql
+++ /dev/null
@@ -1,53 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_03.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-where /*+ indexnl */ similarity-jaccard($a.interests, $b.interests) >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $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
new file mode 100644
index 0000000..bc7b1cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_03.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+where (( /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests) >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
deleted file mode 100644
index 72a8dca..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.aql
+++ /dev/null
@@ -1,55 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
- * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type AddressType as closed {
- number: int32,
- street: string,
- city: string
-}
-
-create type CustomerType as closed {
- cid: int32,
- name: string,
- age: int32?,
- address: AddressType?,
- interests: {{string}},
- children: [ { name: string, age: int32? } ]
-}
-
-create dataset Customers(CustomerType) primary key cid;
-
-create index interests_index on Customers(interests) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ulist-jaccard_04.adm";
-
-for $a in dataset('Customers')
-for $b in dataset('Customers')
-let $jacc := /*+ indexnl */ similarity-jaccard($a.interests, $b.interests)
-where $jacc >= 0.7f and $a.cid < $b.cid
-return {"arec": $a, "brec": $b, "jacc": $jacc }
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
new file mode 100644
index 0000000..77c26cf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ulist-jaccard_04.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, Customers, based on the similarity-jaccard function of its interest sets.
+ * Customers has a keyword index on interests, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+ number : integer,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : integer,
+ name : string,
+ age : integer?,
+ address : AddressType?,
+ interests : {{string}},
+ children : [{
+ name : string,
+ age : integer?
+ }
+]
+};
+
+create dataset Customers(CustomerType) primary key cid;
+
+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
+with jacc as /*+ indexnl */ test.`similarity-jaccard`(a.interests,b.interests)
+where ((jacc >= 0.700000f) and (a.cid < b.cid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 6b1feb6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,59 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..e2158e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_02.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
+ * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ CSX as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index d305ba0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,49 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-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';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..c6c38b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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`;
+
+select element {'arec':a,'brec':b}
+from DBLP as a,
+ DBLP as b
+where ((test.`word-tokens`(a.title) ~= test.`word-tokens`(b.title)) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index 5586919..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /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.
- */
-/*
- * Description : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- * TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- * transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- * multiple indexed nested loop joins of various type of indexes.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-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";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.message-text), word-tokens($t2.message-text), 0.6f)
-where $sim[0] and $t1.tweetid < int64("20") and $t2.tweetid != $t1.tweetid
-return {
- "t1": $t1.tweetid,
- "t2": $t2.tweetid,
- "sim": $sim[1]
-}
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
new file mode 100644
index 0000000..50d2442
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check-after-btree-access.sqlpp
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
+ * TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
+ * transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
+ * multiple indexed nested loop joins of various type of indexes.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+ `screen-name` : string,
+ lang : string,
+ `friends-count` : integer,
+ `statuses-count` : integer,
+ name : string,
+ `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+ tweetid : bigint,
+ user : TwitterUserType,
+ `sender-location` : point,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string,
+ countA : integer,
+ countB : integer
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
+
+create index msgCountAIx on TweetMessages (countA) type btree;
+
+create index msgCountBIx on TweetMessages (countB) type btree;
+
+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
+with sim as test.`similarity-jaccard-check`(test.`word-tokens`(t1.`message-text`),test.`word-tokens`(t2.`message-text`),0.600000f)
+where (sim[0] and (t1.tweetid < test.bigint('20')) and (t2.tweetid != t1.tweetid))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
deleted file mode 100644
index 0a63d12..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.aql
+++ /dev/null
@@ -1,57 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..0c2890a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
+ * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
deleted file mode 100644
index 5c97de1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..fdf6235
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_03.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 : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where (test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
deleted file mode 100644
index 7481ed3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
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
new file mode 100644
index 0000000..8eb16d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_04.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard-check`(test.`word-tokens`(a.title),test.`word-tokens`(b.title),0.500000f)
+where (jacc[0] and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
deleted file mode 100644
index 1b123a6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/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
new file mode 100644
index 0000000..b943eec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_02.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
+ * CSX has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create type test.CSXType as
+ closed {
+ id : integer,
+ csxid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+create dataset CSX(CSXType) primary key id;
+
+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
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
deleted file mode 100644
index e0a2568..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.aql
+++ /dev/null
@@ -1,47 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/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
new file mode 100644
index 0000000..37a119b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_03.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 : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+where ((test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title)) >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
deleted file mode 100644
index f3ca957..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.aql
+++ /dev/null
@@ -1,48 +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.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
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
new file mode 100644
index 0000000..584522a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_04.sqlpp
@@ -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.
+ */
+/*
+ * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
+ * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
+ * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP(DBLPType) primary key id;
+
+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
+with jacc as test.`similarity-jaccard`(test.`word-tokens`(a.title),test.`word-tokens`(b.title))
+where ((jacc >= 0.500000f) and (a.id < b.id))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
deleted file mode 100644
index 65029f3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.aql
+++ /dev/null
@@ -1,100 +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.
- */
-drop dataverse join-super-key_1 if exists;
-
-create dataverse join-super-key_1;
-
-use dataverse join-super-key_1;
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create type PartSuppType as closed {
- ps_partkey: int32,
- ps_suppkey: int32,
- ps_availqty: int32,
- ps_supplycost: double,
- ps_comment: string
-}
-
-
-create nodegroup group1 if not exists on 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 on group1;
-create dataset PartSupp(PartSuppType)
- primary key ps_partkey, ps_suppkey on group1;
-
-
-for $li in dataset('LineItems')
-for $ps in dataset('PartSupp')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and
- $li.l_extendedprice = $ps.ps_supplycost
-return {
- "l_partkey": $li.l_partkey
-}
-
-
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
new file mode 100644
index 0000000..0e7fc0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_01.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 `join-super-key_1` if exists;
+create dataverse `join-super-key_1`;
+
+use `join-super-key_1`;
+
+
+create type `join-super-key_1`.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type `join-super-key_1`.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type `join-super-key_1`.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type `join-super-key_1`.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create type `join-super-key_1`.PartSuppType as
+ closed {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+create nodegroup group1 if not exists on
+ 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 on group1;
+
+create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from LineItems as li,
+ PartSupp as ps
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
deleted file mode 100644
index 831976f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.aql
+++ /dev/null
@@ -1,98 +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.
- */
-drop dataverse join-super-key_01 if exists;
-
-create dataverse join-super-key_01;
-
-use dataverse join-super-key_01;
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create type PartSuppType as closed {
- ps_partkey: int32,
- ps_suppkey: int32,
- ps_availqty: int32,
- ps_supplycost: double,
- ps_comment: string
-}
-
-
-create nodegroup group1 if not exists on 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 on group1;
-create dataset PartSupp(PartSuppType)
- primary key ps_partkey, ps_suppkey on group1;
-
-
-for $ps in dataset('PartSupp')
-for $li in dataset('LineItems')
-where $li.l_partkey = $ps.ps_partkey and $li.l_suppkey = $ps.ps_suppkey and
- $li.l_extendedprice = $ps.ps_supplycost
-return {
- "l_partkey": $li.l_partkey
-}
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
new file mode 100644
index 0000000..7247cdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/join-super-key_02.sqlpp
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 `join-super-key_01` if exists;
+create dataverse `join-super-key_01`;
+
+use `join-super-key_01`;
+
+
+create type `join-super-key_01`.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type `join-super-key_01`.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type `join-super-key_01`.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type `join-super-key_01`.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create type `join-super-key_01`.PartSuppType as
+ closed {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+create nodegroup group1 if not exists on
+ 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 on group1;
+
+create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'l_partkey':li.l_partkey}
+from PartSupp as ps,
+ LineItems as li
+where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
deleted file mode 100644
index ad6833d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.aql
+++ /dev/null
@@ -1,104 +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.
- */
-drop dataverse loj-super-key_01 if exists;
-
-create dataverse loj-super-key_01;
-
-use dataverse loj-super-key_01;
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create type PartSuppType as closed {
- ps_partkey: int32,
- ps_suppkey: int32,
- ps_availqty: int32,
- ps_supplycost: double,
- ps_comment: string
-}
-
-
-create nodegroup group1 if not exists on 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 on group1;
-create dataset PartSupp(PartSuppType)
- primary key ps_partkey, ps_suppkey on group1;
-
-
-for $li in dataset('LineItems')
-let $partsupp :=
- for $ps in dataset('PartSupp')
- where $li.l_partkey = $ps.ps_partkey
- and $li.l_suppkey = $ps.ps_suppkey
- and $li.l_extendedprice = $ps.ps_supplycost
- return $ps
-return {
- "li": $li,
- "partsupp": $partsupp
-}
-
-
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
new file mode 100644
index 0000000..38bd3e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_01.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 `loj-super-key_01` if exists;
+create dataverse `loj-super-key_01`;
+
+use `loj-super-key_01`;
+
+
+create type `loj-super-key_01`.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type `loj-super-key_01`.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type `loj-super-key_01`.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type `loj-super-key_01`.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create type `loj-super-key_01`.PartSuppType as
+ closed {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+create nodegroup group1 if not exists on
+ 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 on group1;
+
+create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'li':li,'partsupp':partsupp}
+from LineItems as li
+with partsupp as (
+ select element ps
+ from PartSupp as ps
+ where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+ )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
deleted file mode 100644
index e0678e9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.aql
+++ /dev/null
@@ -1,104 +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.
- */
-drop dataverse loj-super-key_02 if exists;
-
-create dataverse loj-super-key_02;
-
-use dataverse loj-super-key_02;
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create type PartSuppType as closed {
- ps_partkey: int32,
- ps_suppkey: int32,
- ps_availqty: int32,
- ps_supplycost: double,
- ps_comment: string
-}
-
-
-create nodegroup group1 if not exists on 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 on group1;
-create dataset PartSupp(PartSuppType)
- primary key ps_partkey, ps_suppkey on group1;
-
-
-for $ps in dataset('PartSupp')
-let $items :=
- for $li in dataset('LineItems')
- where $li.l_partkey = $ps.ps_partkey
- and $li.l_suppkey = $ps.ps_suppkey
- and $li.l_extendedprice = $ps.ps_supplycost
- return $li
-return {
- "partsupp": $ps,
- "li": $items
-}
-
-
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
new file mode 100644
index 0000000..9705785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-super-key_02.sqlpp
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 `loj-super-key_02` if exists;
+create dataverse `loj-super-key_02`;
+
+use `loj-super-key_02`;
+
+
+create type `loj-super-key_02`.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type `loj-super-key_02`.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type `loj-super-key_02`.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type `loj-super-key_02`.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create type `loj-super-key_02`.PartSuppType as
+ closed {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+create nodegroup group1 if not exists on
+ 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 on group1;
+
+create dataset PartSupp(PartSuppType) primary key ps_partkey,ps_suppkey on group1;
+
+select element {'partsupp':ps,'li':items}
+from PartSupp as ps
+with items as (
+ select element li
+ from LineItems as li
+ where ((li.l_partkey = ps.ps_partkey) and (li.l_suppkey = ps.ps_suppkey) and (li.l_extendedprice = ps.ps_supplycost))
+ )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
deleted file mode 100644
index f7504ad..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.aql
+++ /dev/null
@@ -1,91 +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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type OrderType as closed {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-create type CustomerType as closed {
- c_custkey: int32,
- c_name: string,
- c_address: string,
- c_nationkey: int32,
- c_phone: string,
- c_acctbal: double,
- c_mktsegment: string,
- c_comment: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset LineItems(LineItemType)
- primary key l_orderkey, l_linenumber on group1;
-create dataset Orders(OrderType)
- primary key o_orderkey on group1;
-create dataset Customers(CustomerType)
- primary key c_custkey on group1;
-
-write output to asterix_nc1:"/tmp/nested_loj.adm";
-
-for $c in dataset('Customers')
-let $orders :=
- for $o in dataset('Orders')
- where $o.o_custkey = $c.c_custkey
- let $items :=
- for $l in dataset('LineItems')
- where $l.l_orderkey = $o.o_orderkey
- return $l
- return {
- "order": $o,
- "items": $items
- }
-return {
- "cust": $c,
- "orders": $orders
-}
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
new file mode 100644
index 0000000..aae756c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested_loj2.sqlpp
@@ -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.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type tpch.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create type tpch.CustomerType as
+ closed {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset LineItems(LineItemType) primary key l_orderkey,l_linenumber on group1;
+
+create dataset Orders(OrderType) primary key o_orderkey on group1;
+
+create dataset Customers(CustomerType) primary key c_custkey on group1;
+
+write output to asterix_nc1:"/tmp/nested_loj.adm";
+select element {'cust':c,'orders':orders}
+from Customers as c
+with orders as (
+ select element {'order':o,'items':items}
+ from Orders as o
+ with items as (
+ select element l
+ from LineItems as l
+ where (l.l_orderkey = o.o_orderkey)
+ )
+ where (o.o_custkey = c.c_custkey)
+ )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
deleted file mode 100644
index 544e715..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.aql
+++ /dev/null
@@ -1,58 +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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
- primary key o_orderkey on group1;
-
-load dataset Orders
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
- $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {
- "o_orderkey": $o.o_orderkey,
- "o_custkey": $o.o_custkey
-}
diff --git a/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
new file mode 100644
index 0000000..190f2f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01.sqlpp
@@ -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.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+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))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
deleted file mode 100644
index 3f9aa3f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.aql
+++ /dev/null
@@ -1,59 +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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as open {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
- primary key o_orderkey on group1;
-
-load dataset Orders
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
- $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {
- "o_orderkey": $o.o_orderkey,
- "o_custkey": $o.o_custkey,
- "o_totalprice": $o.o_totalprice
-}
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
new file mode 100644
index 0000000..64e2395
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02.sqlpp
@@ -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.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+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))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
deleted file mode 100644
index 4eaf437..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.aql
+++ /dev/null
@@ -1,58 +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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
- primary key o_orderkey on group1;
-
-load dataset Orders
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
- $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {
- "o_orderkey": $o.o_orderkey,
- "o_custkey": $o.o_custkey
-}
diff --git a/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
new file mode 100644
index 0000000..4fe9e18
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01.sqlpp
@@ -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.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+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))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
deleted file mode 100644
index b2adee0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.aql
+++ /dev/null
@@ -1,59 +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.
- */
-drop dataverse tpch if exists;
-create dataverse tpch;
-use dataverse tpch;
-
-
-create type OrderType as closed {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Orders(OrderType)
- primary key o_orderkey on group1;
-
-load dataset Orders
-using localfs
-(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
-
-create index idx_Orders_Custkey on Orders(o_custkey);
-
-write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
-
-
-for $o in dataset('Orders')
-where
- $o.o_custkey = 40 and $o.o_totalprice > 150000.0
-order by $o.o_orderkey
-return {
- "o_orderkey": $o.o_orderkey,
- "o_custkey": $o.o_custkey,
- "o_totalprice": $o.o_totalprice
-}
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
new file mode 100644
index 0000000..c16810f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02.sqlpp
@@ -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.
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+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))
+order by o.o_orderkey
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
deleted file mode 100644
index a909b11..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.aql
+++ /dev/null
@@ -1,54 +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.
- */
-drop dataverse pull-select-above-eq-join if exists;
-
-create dataverse pull-select-above-eq-join;
-
-use dataverse pull-select-above-eq-join;
-
-create type UserType as open {
- uid: int32,
- name: string,
- city: string,
- lottery_numbers: {{int32}}
-}
-
-create type VisitorType as open {
- vid: int32,
- name: string,
- city: string,
- lottery_numbers: {{int32}}
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset Users(UserType)
- primary key uid on group1;
-
-create dataset Visitors(VisitorType)
- primary key vid on group1;
-
-
-write output to asterix_nc1:"/tmp/pull-select-above-eq-join.adm";
-
-for $user in dataset('Users')
-for $visitor in dataset('Visitors')
-where $user.name = $visitor.name
- and $user.city != $visitor.city
-return {"uid": $user.uid, "vid": $visitor.vid}
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
new file mode 100644
index 0000000..8634de5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/pull_select_above_eq_join.sqlpp
@@ -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.
+ */
+
+drop dataverse `pull-select-above-eq-join` if exists;
+create dataverse `pull-select-above-eq-join`;
+
+use `pull-select-above-eq-join`;
+
+
+create type `pull-select-above-eq-join`.UserType as
+{
+ uid : integer,
+ name : string,
+ city : string,
+ lottery_numbers : {{integer}}
+};
+
+create type `pull-select-above-eq-join`.VisitorType as
+{
+ vid : integer,
+ name : string,
+ city : string,
+ lottery_numbers : {{integer}}
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset Users(UserType) primary key uid on group1;
+
+create dataset Visitors(VisitorType) primary key vid on 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
+where ((user.name = visitor.name) and (user.city != visitor.city))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
deleted file mode 100644
index 4b4ed34..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.aql
+++ /dev/null
@@ -1,46 +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.
- */
-drop dataverse fuzzyjoin_080 if exists;
-
-create dataverse fuzzyjoin_080;
-
-use dataverse fuzzyjoin_080;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType)
- primary key id on group1;
-
-
-write output to asterix_nc1:'rttest/fuzzyjoin_080.adm';
-
- for $paperDBLP in dataset('DBLP')
- let $matches :=
- for $paper in dataset('DBLP')
- where $paper.authors = $paperDBLP.authors
- return $paper.title
- return {'id': $paperDBLP.id, 'matches':$matches}
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
new file mode 100644
index 0000000..535ae58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/push-project-through-group.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.
+ */
+
+drop dataverse fuzzyjoin_080 if exists;
+create dataverse fuzzyjoin_080;
+
+use fuzzyjoin_080;
+
+
+create type fuzzyjoin_080.DBLPType as
+{
+ id : integer,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create nodegroup group1 if not exists on
+ asterix_nc1,
+ asterix_nc2
+;
+create dataset DBLP(DBLPType) primary key id on group1;
+
+write output to asterix_nc1:"rttest/fuzzyjoin_080.adm";
+select element {'id':paperDBLP.id,'matches':matches}
+from DBLP as paperDBLP
+with matches as (
+ select element paper.title
+ from DBLP as paper
+ where (paper.authors = paperDBLP.authors)
+ )
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
deleted file mode 100644
index 0961e44..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.aql
+++ /dev/null
@@ -1,167 +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.
- */
-drop dataverse q08_group_by if exists;
-
-create dataverse q08_group_by;
-
-use dataverse q08_group_by;
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type OrderType as closed {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-create type CustomerType as closed {
- c_custkey: int32,
- c_name: string,
- c_address: string,
- c_nationkey: int32,
- c_phone: string,
- c_acctbal: double,
- c_mktsegment: string,
- c_comment: string
-}
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type RegionType as closed {
- r_regionkey: int32,
- r_name: string,
- r_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create dataset LineItem(LineItemType)
- primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
- primary key o_orderkey;
-create dataset Customer(CustomerType)
- primary key c_custkey;
-create dataset Supplier(SupplierType)
- primary key s_suppkey;
-create dataset Nation(NationType)
- primary key n_nationkey;
-create dataset Region(RegionType)
- primary key r_regionkey;
-create dataset Part(PartType)
- primary key p_partkey;
-
-for $s in dataset("Supplier")
- for $lnrcop in (
- for $lnrco in (
- for $l in dataset('LineItem')
- for $nrco in (
- for $o in dataset('Orders')
- for $nrc in (
- for $c in dataset('Customer')
- for $nr in (
- for $n1 in dataset('Nation')
- for $r1 in dataset('Region')
- where $n1.n_regionkey = $r1.r_regionkey and $r1.r_name = 'AMERICA'
- return { "n_nationkey": $n1.n_nationkey }
- )
- where $c.c_nationkey = $nr.n_nationkey
- return { "c_custkey": $c.c_custkey }
- )
- where $nrc.c_custkey = $o.o_custkey
- return {
- "o_orderdate" : $o.o_orderdate,
- "o_orderkey": $o.o_orderkey
- }
- )
- where $l.l_orderkey = $nrco.o_orderkey
- and $nrco.o_orderdate >= '1995-01-01'
- and $nrco.o_orderdate <= '1996-12-31'
- return {
- "o_orderdate": $nrco.o_orderdate,
- "l_partkey": $l.l_partkey,
- "l_discount": $l.l_discount,
- "l_extendedprice": $l.l_extendedprice,
- "l_suppkey": $l.l_suppkey
- }
- )
- for $p in dataset("Part")
- where $p.p_partkey = $lnrco.l_partkey and $p.p_type = 'ECONOMY ANODIZED STEEL'
- return {
- "o_orderdate": $lnrco.o_orderdate,
- "l_discount": $lnrco.l_discount,
- "l_extendedprice": $lnrco.l_extendedprice,
- "l_suppkey": $lnrco.l_suppkey
- }
- )
- where $s.s_suppkey = $lnrcop.l_suppkey
- return {
- "o_orderdate": $lnrcop.o_orderdate,
- "l_discount": $lnrcop.l_discount,
- "l_extendedprice": $lnrcop.l_extendedprice,
- "l_suppkey": $lnrcop.l_suppkey,
- "s_nationkey": $s.s_nationkey
- }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
new file mode 100644
index 0000000..4ff3d31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q08_group_by.sqlpp
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 q08_group_by if exists;
+create dataverse q08_group_by;
+
+use q08_group_by;
+
+
+create type q08_group_by.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type q08_group_by.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create type q08_group_by.CustomerType as
+ closed {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+create type q08_group_by.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type q08_group_by.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type q08_group_by.RegionType as
+ closed {
+ r_regionkey : integer,
+ r_name : string,
+ r_comment : string
+};
+
+create type q08_group_by.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create dataset Orders(OrderType) primary key o_orderkey;
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+create dataset Supplier(SupplierType) primary key s_suppkey;
+
+create dataset Nation(NationType) primary key n_nationkey;
+
+create dataset Region(RegionType) primary key r_regionkey;
+
+create dataset Part(PartType) primary key p_partkey;
+
+select element {'o_orderdate':lnrcop.o_orderdate,'l_discount':lnrcop.l_discount,'l_extendedprice':lnrcop.l_extendedprice,'l_suppkey':lnrcop.l_suppkey,'s_nationkey':s.s_nationkey}
+from Supplier as s,
+ (
+ select element {'o_orderdate':lnrco.o_orderdate,'l_discount':lnrco.l_discount,'l_extendedprice':lnrco.l_extendedprice,'l_suppkey':lnrco.l_suppkey}
+ from (
+ select element {'o_orderdate':nrco.o_orderdate,'l_partkey':l.l_partkey,'l_discount':l.l_discount,'l_extendedprice':l.l_extendedprice,'l_suppkey':l.l_suppkey}
+ from LineItem as l,
+ (
+ select element {'o_orderdate':o.o_orderdate,'o_orderkey':o.o_orderkey}
+ from Orders as o,
+ (
+ select element {'c_custkey':c.c_custkey}
+ from Customer as c,
+ (
+ select element {'n_nationkey':n1.n_nationkey}
+ from Nation as n1,
+ Region as r1
+ where ((n1.n_regionkey = r1.r_regionkey) and (r1.r_name = 'AMERICA'))
+ ) as nr
+ where (c.c_nationkey = nr.n_nationkey)
+ ) as nrc
+ where (nrc.c_custkey = o.o_custkey)
+ ) as nrco
+ where ((l.l_orderkey = nrco.o_orderkey) and (nrco.o_orderdate >= '1995-01-01') and (nrco.o_orderdate <= '1996-12-31'))
+ ) as lnrco,
+ Part as p
+ where ((p.p_partkey = lnrco.l_partkey) and (p.p_type = 'ECONOMY ANODIZED STEEL'))
+) as lnrcop
+where (s.s_suppkey = lnrcop.l_suppkey)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
deleted file mode 100644
index adb6efe..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.aql
+++ /dev/null
@@ -1,171 +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.
- */
-drop dataverse q09_group_by if exists;
-
-create dataverse q09_group_by;
-
-use dataverse q09_group_by;
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: int32,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create type OrderType as closed {
- o_orderkey: int32,
- o_custkey: int32,
- o_orderstatus: string,
- o_totalprice: double,
- o_orderdate: string,
- o_orderpriority: string,
- o_clerk: string,
- o_shippriority: int32,
- o_comment: string
-}
-
-create type CustomerType as closed {
- c_custkey: int32,
- c_name: string,
- c_address: string,
- c_nationkey: int32,
- c_phone: string,
- c_acctbal: double,
- c_mktsegment: string,
- c_comment: string
-}
-
-create type SupplierType as closed {
- s_suppkey: int32,
- s_name: string,
- s_address: string,
- s_nationkey: int32,
- s_phone: string,
- s_acctbal: double,
- s_comment: string
-}
-
-create type NationType as closed {
- n_nationkey: int32,
- n_name: string,
- n_regionkey: int32,
- n_comment: string
-}
-
-create type RegionType as closed {
- r_regionkey: int32,
- r_name: string,
- r_comment: string
-}
-
-create type PartType as closed {
- p_partkey: int32,
- p_name: string,
- p_mfgr: string,
- p_brand: string,
- p_type: string,
- p_size: int32,
- p_container: string,
- p_retailprice: double,
- p_comment: string
-}
-
-create type PartSuppType as closed {
- ps_partkey: int32,
- ps_suppkey: int32,
- ps_availqty: int32,
- ps_supplycost: double,
- ps_comment: string
-}
-
-create dataset LineItem(LineItemType)
- primary key l_orderkey, l_linenumber;
-create dataset Orders(OrderType)
- primary key o_orderkey;
-create dataset Supplier(SupplierType)
- primary key s_suppkey;
-create dataset Region(RegionType)
- primary key r_regionkey;
-create dataset Nation(NationType)
- primary key n_nationkey;
-create dataset Part(PartType)
- primary key p_partkey;
-create dataset Partsupp(PartSuppType)
- primary key ps_partkey, ps_suppkey;
-create dataset Customer(CustomerType)
- primary key c_custkey;
-
-for $p in dataset('Part')
- for $l2 in (
- for $ps in dataset('Partsupp')
- for $l1 in (
- for $s1 in (
- for $s in dataset('Supplier')
- for $n in dataset('Nation')
- where $n.n_nationkey = $s.s_nationkey
- return {
- "s_suppkey": $s.s_suppkey,
- "n_name": $n.n_name
- }
- )
- for $l in dataset('LineItem')
- where $s1.s_suppkey = $l.l_suppkey
- return {
- "l_suppkey": $l.l_suppkey,
- "l_extendedprice": $l.l_extendedprice,
- "l_discount": $l.l_discount,
- "l_quantity": $l.l_quantity,
- "l_partkey": $l.l_partkey,
- "l_orderkey": $l.l_orderkey,
- "n_name": $s1.n_name
- }
- )
- where $ps.ps_suppkey = $l1.l_suppkey and $ps.ps_partkey = $l1.l_partkey
- return {
- "l_extendedprice": $l1.l_extendedprice,
- "l_discount": $l1.l_discount,
- "l_quantity": $l1.l_quantity,
- "l_partkey": $l1.l_partkey,
- "l_orderkey": $l1.l_orderkey,
- "n_name": $l1.n_name,
- "ps_supplycost": $ps.ps_supplycost
- }
- )
- where contains($p.p_name, 'green') and $p.p_partkey = $l2.l_partkey
- return {
- "l_extendedprice": $l2.l_extendedprice,
- "l_discount": $l2.l_discount,
- "l_quantity": $l2.l_quantity,
- "l_orderkey": $l2.l_orderkey,
- "n_name": $l2.n_name,
- "ps_supplycost": $l2.ps_supplycost
- }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
new file mode 100644
index 0000000..44f4760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q09_group_by.sqlpp
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 q09_group_by if exists;
+create dataverse q09_group_by;
+
+use q09_group_by;
+
+
+create type q09_group_by.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : integer,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create type q09_group_by.OrderType as
+ closed {
+ o_orderkey : integer,
+ o_custkey : integer,
+ o_orderstatus : string,
+ o_totalprice : double,
+ o_orderdate : string,
+ o_orderpriority : string,
+ o_clerk : string,
+ o_shippriority : integer,
+ o_comment : string
+};
+
+create type q09_group_by.CustomerType as
+ closed {
+ c_custkey : integer,
+ c_name : string,
+ c_address : string,
+ c_nationkey : integer,
+ c_phone : string,
+ c_acctbal : double,
+ c_mktsegment : string,
+ c_comment : string
+};
+
+create type q09_group_by.SupplierType as
+ closed {
+ s_suppkey : integer,
+ s_name : string,
+ s_address : string,
+ s_nationkey : integer,
+ s_phone : string,
+ s_acctbal : double,
+ s_comment : string
+};
+
+create type q09_group_by.NationType as
+ closed {
+ n_nationkey : integer,
+ n_name : string,
+ n_regionkey : integer,
+ n_comment : string
+};
+
+create type q09_group_by.RegionType as
+ closed {
+ r_regionkey : integer,
+ r_name : string,
+ r_comment : string
+};
+
+create type q09_group_by.PartType as
+ closed {
+ p_partkey : integer,
+ p_name : string,
+ p_mfgr : string,
+ p_brand : string,
+ p_type : string,
+ p_size : integer,
+ p_container : string,
+ p_retailprice : double,
+ p_comment : string
+};
+
+create type q09_group_by.PartSuppType as
+ closed {
+ ps_partkey : integer,
+ ps_suppkey : integer,
+ ps_availqty : integer,
+ ps_supplycost : double,
+ ps_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create dataset Orders(OrderType) primary key o_orderkey;
+
+create dataset Supplier(SupplierType) primary key s_suppkey;
+
+create dataset Region(RegionType) primary key r_regionkey;
+
+create dataset Nation(NationType) primary key n_nationkey;
+
+create dataset Part(PartType) primary key p_partkey;
+
+create dataset Partsupp(PartSuppType) primary key ps_partkey,ps_suppkey;
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+select element {'l_extendedprice':l2.l_extendedprice,'l_discount':l2.l_discount,'l_quantity':l2.l_quantity,'l_orderkey':l2.l_orderkey,'n_name':l2.n_name,'ps_supplycost':l2.ps_supplycost}
+from Part as p,
+ (
+ select element {'l_extendedprice':l1.l_extendedprice,'l_discount':l1.l_discount,'l_quantity':l1.l_quantity,'l_partkey':l1.l_partkey,'l_orderkey':l1.l_orderkey,'n_name':l1.n_name,'ps_supplycost':ps.ps_supplycost}
+ from Partsupp as ps,
+ (
+ select element {'l_suppkey':l.l_suppkey,'l_extendedprice':l.l_extendedprice,'l_discount':l.l_discount,'l_quantity':l.l_quantity,'l_partkey':l.l_partkey,'l_orderkey':l.l_orderkey,'n_name':s1.n_name}
+ from (
+ select element {'s_suppkey':s.s_suppkey,'n_name':n.n_name}
+ from Supplier as s,
+ Nation as n
+ where (n.n_nationkey = s.s_nationkey)
+ ) as s1,
+ LineItem as l
+ where (s1.s_suppkey = l.l_suppkey)
+ ) as l1
+ where ((ps.ps_suppkey = l1.l_suppkey) and (ps.ps_partkey = l1.l_partkey))
+) as l2
+where (q09_group_by.contains(p.p_name,'green') and (p.p_partkey = l2.l_partkey))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
deleted file mode 100644
index 827b02a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.aql
+++ /dev/null
@@ -1,58 +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.
- */
-/*
- * Description : This test case is to verify the fix for issue601
- * https://code.google.com/p/asterixdb/issues/detail?id=601
- * Expected Res : SUCCESS
- * Date : 10th Oct 2014
- */
-
-drop dataverse tpch if exists;
-create dataverse tpch;
-
-use dataverse tpch;
-
-create type LineItemType as closed {
- l_orderkey: int32,
- l_partkey: int32,
- l_suppkey: int32,
- l_linenumber: int32,
- l_quantity: double,
- l_extendedprice: double,
- l_discount: double,
- l_tax: double,
- l_returnflag: string,
- l_linestatus: string,
- l_shipdate: string,
- l_commitdate: string,
- l_receiptdate: string,
- l_shipinstruct: string,
- l_shipmode: string,
- l_comment: string
-}
-
-create dataset LineItem(LineItemType)
- primary key l_orderkey, l_linenumber;
-
-for $l in dataset('LineItem')
-group by $l_linenumber := $l.l_linenumber with $l
-return {
- "l_linenumber": $l_linenumber,
- "count_order": count($l)
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
new file mode 100644
index 0000000..a5861be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue601.sqlpp
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : This test case is to verify the fix for issue601
+ * https://code.google.com/p/asterixdb/issues/detail?id=601
+ * Expected Res : SUCCESS
+ * Date : 10th Oct 2014
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+ l_orderkey : integer,
+ l_partkey : integer,
+ l_suppkey : integer,
+ l_linenumber : integer,
+ l_quantity : double,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+select element {'l_linenumber':l_linenumber,'count_order':count(l)}
+from LineItem as l
+group by l.l_linenumber as l_linenumber
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
deleted file mode 100644
index 18acdf0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.aql
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in dataset s
-for $y in (
- for $z in {{ {"a":1, "c":1},{"a":2, "c":2},{"a":1, "c":null} }} where $x.b=$z.a
- return $z.c
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.sqlpp
new file mode 100644
index 0000000..ee21bc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849-2.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 : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date : 2nd Feb. 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+ b : integer
+};
+
+create dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from s as x,
+ (
+ select element z.c
+ from {{{'a':1,'c':1},{'a':2,'c':2},{'a':1,'c':null}}} as z
+ where (x.b = z.a)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
deleted file mode 100644
index 3e2bea3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.aql
+++ /dev/null
@@ -1,42 +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.
- */
-/*
- * Description : This test case is to verify the fix for issue827
- * https://code.google.com/p/asterixdb/issues/detail?id=849
- * Expected Res : SUCCESS
- * Date : 2nd Feb. 2015
- */
-
-drop dataverse test if exists;
-create dataverse test;
-
-use dataverse test;
-
-create type sType as closed{b : int32};
-create dataset s(sType) primary key b;
-
-insert into dataset s ({ "b" : 1});
-insert into dataset s ({ "b" : 3});
-
-for $x in {{ {"a":1},{"a":2} }}
-for $y in (
- for $z in dataset s where $x.a=$z.b
- return $z.b
-)
-return {"x":$x,"y":$y}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.sqlpp
new file mode 100644
index 0000000..5b0e7fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query_issue849.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 : This test case is to verify the fix for issue827
+ * https://code.google.com/p/asterixdb/issues/detail?id=849
+ * Expected Res : SUCCESS
+ * Date : 2nd Feb. 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.sType as
+ closed {
+ b : integer
+};
+
+create dataset s(sType) primary key b;
+
+insert into s
+select element {'b':1};
+insert into s
+select element {'b':3};
+select element {'x':x,'y':y}
+from {{{'a':1},{'a':2}}} as x,
+ (
+ select element z.b
+ from s as z
+ where (x.a = z.b)
+) as y
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
deleted file mode 100644
index 2b6df8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.aql
+++ /dev/null
@@ -1,71 +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.
- */
-/*
- * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue : 730, 741
- * Expected Res : Success
- * Date : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-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";
-
-for $t1 in dataset('TweetMessages')
-let $n := create-circle($t1.sender-location, 0.5)
-where $t1.tweetid < int64("10")
-order by $t1.tweetid
-return {
-"tweetid1": $t1.tweetid,
-"loc1":$t1.sender-location,
-"nearby-message": for $t2 in dataset('TweetMessages')
- where spatial-intersect($t2.sender-location, $n) and $t1.tweetid != $t2.tweetid
- order by $t2.tweetid
- return {"tweetid2":$t2.tweetid, "loc2":$t2.sender-location}
-};
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
new file mode 100644
index 0000000..a30f22c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.sqlpp
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue : 730, 741
+ * Expected Res : Success
+ * Date : 8th May 2014
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+ `screen-name` : string,
+ lang : string,
+ `friends-count` : integer,
+ `statuses-count` : integer,
+ name : string,
+ `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+ tweetid : bigint,
+ user : TwitterUserType,
+ `sender-location` : point,
+ `send-time` : datetime,
+ `referred-topics` : {{string}},
+ `message-text` : string,
+ countA : integer,
+ countB : integer
+};
+
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create index twmSndLocIx on TweetMessages (`sender-location`) type rtree;
+
+create index msgCountAIx on TweetMessages (countA) type btree;
+
+create index msgCountBIx on TweetMessages (countB) type btree;
+
+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
+ where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+ order by t2.tweetid
+ )}
+from TweetMessages as t1
+with n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
deleted file mode 100644
index dcbf53f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.aql
+++ /dev/null
@@ -1,51 +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.
- */
-/*
- * Description : Joins two datasets on the intersection of their point attributes.
- * The dataset 'MyData2' has an RTree index, and we expect the
- * join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
- id: int32,
- point: point,
- kwds: string,
- line1: line,
- line2: line,
- poly1: polygon,
- poly2: polygon,
- rec: rectangle
-}
-
-create dataset MyData1(MyRecord) primary key id;
-create dataset MyData2(MyRecord) primary key id;
-
-create index rtree_index on MyData2(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_02.adm";
-
-for $a in dataset('MyData1')
-for $b in dataset('MyData2')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $b}
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
new file mode 100644
index 0000000..f87832e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_02.sqlpp
@@ -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.
+ */
+/*
+ * Description : Joins two datasets on the intersection of their point attributes.
+ * The dataset 'MyData2' has an RTree index, and we expect the
+ * join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+ id : integer,
+ point : point,
+ kwds : string,
+ line1 : line,
+ line2 : line,
+ poly1 : polygon,
+ poly2 : polygon,
+ rec : rectangle
+};
+
+create dataset MyData1(MyRecord) primary key id;
+
+create dataset MyData2(MyRecord) primary key id;
+
+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
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
deleted file mode 100644
index 624fe9e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.aql
+++ /dev/null
@@ -1,50 +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.
- */
-/*
- * Description : Self-joins a dataset on the intersection of its point attribute.
- * The dataset has an RTree index, and we expect the
- * join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type MyRecord as closed {
- id: int32,
- point: point,
- kwds: string,
- line1: line,
- line2: line,
- poly1: polygon,
- poly2: polygon,
- rec: rectangle
-}
-
-create dataset MyData(MyRecord) primary key id;
-
-create index rtree_index on MyData(point) type rtree;
-
-write output to asterix_nc1:"rttest/rtree-index-join_spatial-intersect-point_03.adm";
-
-for $a in dataset('MyData')
-for $b in dataset('MyData')
-where spatial-intersect($a.point, $b.point)
-return {"a": $a, "b": $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
new file mode 100644
index 0000000..afe7a72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/spatial-intersect-point_03.sqlpp
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 : Self-joins a dataset on the intersection of its point attribute.
+ * The dataset has an RTree index, and we expect the
+ * join to be transformed into an indexed nested-loop join.
+ * Success : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.MyRecord as
+ closed {
+ id : integer,
+ point : point,
+ kwds : string,
+ line1 : line,
+ line2 : line,
+ poly1 : polygon,
+ poly2 : polygon,
+ rec : rectangle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+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
+where test.`spatial-intersect`(a.point,b.point)
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
deleted file mode 100644
index b8c7a8a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.aql
+++ /dev/null
@@ -1,43 +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.
- */
-/*
- * Description : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
- * Expected Res : Success
- * Date : 21th December 2013
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
-
-create type TestType as open {
- id : int32,
- fname : string,
- lname : string
-}
-
-create dataset testdst(TestType) primary key id;
-
-create index sec_Idx on testdst(fname);
-
-for $emp in dataset('testdst')
-where $emp.fname /*+ skip-index */ >= "Max" and $emp.fname <= "Roger"
-return $emp
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
new file mode 100644
index 0000000..6235842
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-2.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.
+ */
+/*
+ * Description : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
+ * Expected Res : Success
+ * Date : 21th December 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+write output to asterix_nc1:"rttest/btree-index_btree-secondary-57.adm";
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where ((emp.fname /*+ skip-index */ >= 'Max') and (emp.fname <= 'Roger'))
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
deleted file mode 100644
index a3bb0fc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.aql
+++ /dev/null
@@ -1,49 +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.
- */
-drop dataverse TinySocial if exists;
-create dataverse TinySocial;
-use dataverse TinySocial;
-
-
-create type EmploymentType as open {
- organization-name: string,
- start-date: date,
- end-date: date?
-}
-
-create type FacebookUserType as closed {
- id: int32,
- alias: string,
- name: string,
- user-since: datetime,
- friend-ids: {{ int32 }},
- employment: [EmploymentType]
-}
-
-create dataset FacebookUsers(FacebookUserType)
-primary key id;
-
-
-let $lonelyusers := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-let $lonelyusers2 := for $d in dataset FacebookUsers where count($d.friend-ids) < 2 return $d
-for $l1 in $lonelyusers
-for $l2 in $lonelyusers2
-where $l1.id < $l2.id
-order by $l1.id, $l2.id
-return { "user1": { "id": $l1.id, "name": $l1.name }, "user2": { "id": $l2.id, "name": $l2.name } };
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
new file mode 100644
index 0000000..a32530a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 TinySocial if exists;
+create dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.EmploymentType as
+{
+ `organization-name` : string,
+ `start-date` : date,
+ `end-date` : date?
+};
+
+create type TinySocial.FacebookUserType as
+ closed {
+ id : integer,
+ alias : string,
+ name : string,
+ `user-since` : datetime,
+ `friend-ids` : {{integer}},
+ employment : [EmploymentType]
+};
+
+create dataset FacebookUsers(FacebookUserType) primary key id;
+
+with lonelyusers as (
+ select element d
+ from FacebookUsers as d
+ where (TinySocial.coll_count(d.`friend-ids`) < 2)
+ ),
+ lonelyusers2 as (
+ select element d
+ from FacebookUsers as d
+ where (TinySocial.coll_count(d.`friend-ids`) < 2)
+ )
+select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
+from lonelyusers as l1,
+ lonelyusers2 as l2
+where (l1.id < l2.id)
+order by l1.id,l2.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
index 07fde95..586d5fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02.plan
@@ -2,16 +2,16 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$34(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$24] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$52(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$42] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$24(ASC), $$25(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$24] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC), $$43(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -19,7 +19,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$57(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
index 8e1be33..d473d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
@@ -6,8 +6,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$26] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index 4173afc..aeaf88f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$45(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -8,7 +8,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index 77aba01..62a8abd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -8,7 +8,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
index ef8a923..a050d80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index 001e10b..ef8a923 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
index e12cdd8..4b4412e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
index d7667d6..bebed65 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-60.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
index 4b4412e..577c44b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-61.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
index efd83eee..04b9728 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-62.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
index faeb779..bb57fa7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-63.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$21(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
index 116ff8a..180edb2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$10][$$11] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$26][$$27] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
index 5cb092c..1e82aa4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
@@ -1,11 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$6] |PARTITIONED|
+ -- SORT_GROUP_BY[$$27] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$6] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
index b37868e..31b1af9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
@@ -3,22 +3,22 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$16] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$77] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$16][$$18] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$77][$$79] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$18] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$79] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
index f4a21e2..4deb445 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/disjunction-to-join-delete-3.plan
@@ -12,7 +12,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$9][$$11] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$17][$$19] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
new file mode 100644
index 0000000..d38f1d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/distinct_aggregate.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$102, $$103] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$102, $$103] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$l_returnflag, $$l_linestatus] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$92, $$93, $$94] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$92, $$93, $$94] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
index d3510b3..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-dblp-csx.plan
@@ -1,93 +0,0 @@
--- DISTRIBUTE_RESULT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$18(ASC), $$19(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$42, $$44] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC), $$44(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$42, $$44] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$8][$$17] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$8] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$46] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$46(ASC), $$5(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$46] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$3][$$26] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$3] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |UNPARTITIONED|
- -- ASSIGN |UNPARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$47] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC), $$14(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$12][$$35] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- REPLICATE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
index 93fe350..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/fj-phase2-with-hints.plan
@@ -1,57 +0,0 @@
--- DISTRIBUTE_RESULT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$31(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$27] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- IN_MEMORY_STABLE_SORT [$$4(ASC)] |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- IN_MEMORY_HASH_JOIN [$$2][$$7] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- BROADCAST_EXCHANGE |PARTITIONED|
- -- RUNNING_AGGREGATE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$28(ASC), $$7(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$28(ASC), $$7(ASC)] |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$36] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$6] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
index e11e2a8..855fd33 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -4,7 +4,7 @@
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- MATERIALIZE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$6] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
index 08635f7..75444af 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/introhashpartitionmerge.plan
@@ -1,11 +1,11 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$36(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$14][$$15] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -13,7 +13,7 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$13(ASC)] HASH:[$$15] |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$36(ASC)] HASH:[$$38] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic.plan
@@ -1,7 +1,7 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$5(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
index 5aad07e..01f4b08 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains.plan
@@ -1,12 +1,12 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$5(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
index c8aaaabc..6037e0c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$7(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic.plan
@@ -1,7 +1,7 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$7(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check.plan
@@ -1,12 +1,12 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$7(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$11(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
index 6139d4c..7b7be55 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic.plan
@@ -1,7 +1,7 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$7(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
index 478d8bb..2a0e3fe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance.plan
@@ -1,12 +1,12 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$7(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$15(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$11(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
index a1426f3..2785605 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance.plan
@@ -1,12 +1,12 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$6(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
index ee235ae..38ca6cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$8(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
index b16fcf8..a8c1364 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains.plan
@@ -1,7 +1,7 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$5(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$13(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
index 362a1c4..64c0782 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$17(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
index 0a243c7..31b84a2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$18(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 323f681..1fc3fc0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$14(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
index 362a1c4..fccc3c7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$9(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$24(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
index ff74295..cafb02c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-substring.plan
@@ -8,7 +8,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$13(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$11(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
index 503fd28..96493a3 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic.plan
@@ -1,7 +1,7 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$8(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
index 6194e08..19ce662 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let.plan
@@ -1,12 +1,12 @@
-- DISTRIBUTE_RESULT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$8(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
index 0a243c7..4c0c9e1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$10(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
index 9748866..38dced9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -5,7 +5,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$11(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index 0cb0a86..2ef2633 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -9,7 +9,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$16(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$46(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 3e4c9bc..11cbac1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -13,7 +13,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 736928c..de31517 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -12,7 +12,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$20] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$43] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -12,7 +12,7 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 3e4c9bc..11cbac1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -13,7 +13,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 5133569..74fc277 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -11,7 +11,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 5133569..74fc277 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -11,7 +11,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index d1a248a..9afe4d7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$20] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$43] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -12,7 +12,7 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$43] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -22,7 +22,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 5b08bf5..5134f39 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
@@ -3,18 +3,18 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$25] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$51] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$25] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$36][$$25] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$62][$$51] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
@@ -24,14 +24,14 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$25] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$51] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
index 7065d9e..97c2b45 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01.plan
@@ -2,19 +2,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$27] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$63(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$53] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC), $$54(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$27] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$69][$$53] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -25,7 +25,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -36,7 +36,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$77(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
index 5668704..8f1ca57 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
@@ -2,19 +2,19 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$37(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$27] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$63(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$53] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
+ -- STABLE_SORT [$$53(ASC), $$54(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$43][$$27] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$69][$$53] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -25,7 +25,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$53] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -33,7 +33,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$73(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index 7465d81..8c0aec1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$38] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index 968246d..0d63ee1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$37] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$24][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$40][$$30] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$24][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$40][$$30] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$17] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$40] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$16] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$39] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$16] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index 7465d81..8c0aec1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$38] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index 8d0f7f8..0022f1c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$26(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index 968246d..0d63ee1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$37] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$54(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index a261ab7..8a8090e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- UNION_ALL |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$25(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$41(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index d667093..8f3d988 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$38] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index 509149f..d5e5925 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$37] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 80b2a89..2bb6f2d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$20][$$12] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$36][$$28] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$28] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$38(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index d667093..8f3d988 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$38] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index f6e6db1..810a139 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21][$$13] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$37][$$29] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$29] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$23(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$39(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index 509149f..d5e5925 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$26][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$49][$$37] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$37] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -18,7 +18,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$51(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$24][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$40][$$30] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 420d3ea..3ac5428 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$24][$$14] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$40][$$30] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$30] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$43(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
index 0cd8363..53c00eb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$37][$$24] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$60][$$47] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$24] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$47] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -26,7 +26,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$42(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$65(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 3b0a3c9..f939cb1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$17] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$40] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 5b169eba..9d7c9bc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$25][$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$41][$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$31] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$44(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index 443e560..0ea5cc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -4,12 +4,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$29][$$16] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$52][$$39] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$16] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -17,7 +17,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$32(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$55(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH |PARTITIONED|
-- BROADCAST_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
index 64eb060..d6e272b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
@@ -4,8 +4,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$17, $$22, $$24][$$19, $$23, $$20] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17, $$24] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$33, $$38, $$40][$$35, $$39, $$36] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$33, $$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
index 92adb3d..49e2b90 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$17, $$23, $$18][$$19, $$22, $$24] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$33, $$39, $$34][$$35, $$38, $$40] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -12,7 +12,7 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$19, $$24] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35, $$40] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
index d79e1f2..12abcc6 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
@@ -4,19 +4,19 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$17, $$18] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$40, $$41] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$17(ASC), $$18(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17, $$18] |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC), $$41(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40, $$41] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$21, $$17, $$25][$$22, $$19, $$20] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$17, $$25] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$44, $$40, $$48][$$45, $$42, $$43] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40, $$48] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
index c1cd283..a82ff05 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
@@ -4,25 +4,25 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$17, $$18] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$40, $$41] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$17(ASC), $$18(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$40(ASC), $$41(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$22, $$17, $$18][$$21, $$19, $$25] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$45, $$40, $$41][$$44, $$42, $$48] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$19, $$25] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$42, $$48] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
index 08b156e..3bb5c88 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
@@ -4,10 +4,10 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$18] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$56] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
- -- MICRO_PRE_CLUSTERED_GROUP_BY[$$19] |LOCAL|
+ -- MICRO_PRE_CLUSTERED_GROUP_BY[$$57] |LOCAL|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
@@ -17,24 +17,24 @@
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$18(ASC), $$19(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$18] |PARTITIONED|
+ -- STABLE_SORT [$$56(ASC), $$57(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$56] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$19][$$20] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$19] |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$18][$$22] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$57][$$58] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$57] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$56][$$60] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$22] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$58] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
index e0a20c8..59dc8e8 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$21(ASC) ] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$28(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
index c30de53..19be1fa 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02.plan
@@ -15,14 +15,14 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$14(ASC) ] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$23(ASC) ] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$29(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
index b0d8e64..8d5343d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
@@ -6,15 +6,15 @@
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$18][$$19] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$18] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$34][$$35] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$34] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$19] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$35] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
index 3cc71e0..8d02861 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
@@ -3,26 +3,26 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$12] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$36] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$12(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$12] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$36] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$15][$$14] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$39][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$14] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$38] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
index 1aca471..db613d0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q08_group_by.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$78][$$106] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$169][$$197] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -12,15 +12,15 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$106] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$197] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$103][$$85] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$103] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$194][$$176] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$194] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$79][$$81] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$79] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$170][$$172] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$170] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -28,11 +28,11 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$172] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$94][$$82] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$94] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$185][$$173] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$185] |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -40,22 +40,22 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$82] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$173] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$91][$$83] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$91] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$182][$$174] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$182] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$174] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$89][$$84] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$180][$$175] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$180] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
index 9f48723..d6da05e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q09_group_by.plan
@@ -4,7 +4,7 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$62][$$84] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$123][$$145] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
@@ -12,10 +12,10 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$84] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$145] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$63, $$64][$$84, $$69] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$124, $$125][$$145, $$130] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
@@ -23,15 +23,15 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$84, $$69] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$145, $$130] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$65][$$69] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$65] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$126][$$130] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$126] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$72][$$66] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$72] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$133][$$127] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$133] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
@@ -45,7 +45,7 @@
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$69] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$130] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
index 369f475..e65a903 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue601.plan
@@ -3,13 +3,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- SORT_GROUP_BY[$$13] |PARTITIONED|
+ -- SORT_GROUP_BY[$$40] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$13] |PARTITIONED|
- -- SORT_GROUP_BY[$$10] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40] |PARTITIONED|
+ -- SORT_GROUP_BY[$$37] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
index 8908448..8b7d56e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849-2.plan
@@ -20,12 +20,12 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$14][$$16] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$38][$$40] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$16] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$40] |PARTITIONED|
-- STREAM_PROJECT |UNPARTITIONED|
-- ASSIGN |UNPARTITIONED|
-- UNNEST |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
index dd7b473..709d5d5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query_issue849.plan
@@ -20,8 +20,8 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$15][$$14] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$39][$$38] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$39] |PARTITIONED|
-- ASSIGN |UNPARTITIONED|
-- UNNEST |UNPARTITIONED|
-- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
index 3bc6933..80f1234 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02.plan
@@ -2,16 +2,16 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$35(ASC) ] |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$27] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$60(ASC) ] |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$52] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- STREAM_SELECT |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$27(ASC), $$28(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$27] |PARTITIONED|
+ -- STABLE_SORT [$$52(ASC), $$53(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$52] |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- STREAM_SELECT |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,7 +20,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$47(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
@@ -8,7 +8,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
index 64fdc8c..3229470 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
@@ -8,7 +8,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
index 807e509..0d9a257 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-2.plan
@@ -7,7 +7,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$11(ASC)] |PARTITIONED|
+ -- STABLE_SORT [$$19(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
index dab5a7b..307af54 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization.plan
@@ -2,8 +2,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- SORT_MERGE_EXCHANGE [$$30(ASC), $$31(ASC) ] |PARTITIONED|
- -- STABLE_SORT [$$30(ASC), $$31(ASC)] |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$74(ASC), $$75(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$74(ASC), $$75(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- NESTED_LOOP |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
index e9fafd7..e69de29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
@@ -1,33 +0,0 @@
--- DISTRIBUTE_RESULT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- PRE_CLUSTERED_GROUP_BY[$$20] |PARTITIONED|
- {
- -- AGGREGATE |LOCAL|
- -- STREAM_SELECT |LOCAL|
- -- NESTED_TUPLE_SOURCE |LOCAL|
- }
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC), $$18(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- HYBRID_HASH_JOIN [$$3][$$22] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$3] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- UNNEST |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$22] |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
index f65ede3..da7ba31 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
@@ -35,5 +35,10 @@
<output-dir compare="Text">secondary-index</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="metrics">
+ <compilation-unit name="secondary-index-index-only">
+ <output-dir compare="Text">secondary-index-index-only</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
+++ /dev/null
@@ -1,20 +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.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.uri
@@ -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.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
deleted file mode 100644
index bcc0edc..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
+++ /dev/null
@@ -1,22 +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.
- */
-
-//polltimeoutsecs=10
-
-/query/service/status/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
new file mode 100644
index 0000000..286c097
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.uri
@@ -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.
+ */
+
+//polltimeoutsecs=10
+// handlevariable=handle
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
deleted file mode 100644
index 8417a7e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
+++ /dev/null
@@ -1,20 +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.
- */
-
-/query/service/result/$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
new file mode 100644
index 0000000..a955fed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.uri
@@ -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.
+ */
+
+$handle
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
new file mode 100644
index 0000000..b59be3b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int16/divide_int16.2.query.aql
@@ -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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c2 div $c1,"result2": $c2 div $c2,"result3": $c2 div $c3,"result4": $c2 div $c4,"result5": $c2 div $c5, "result6": $c2 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
new file mode 100644
index 0000000..d8ad04a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int32/divide_int32.2.query.aql
@@ -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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c3 div $c1,"result2": $c3 div $c2,"result3": $c3 div $c3,"result4": $c3 div $c4,"result5": $c3 div $c5, "result6": $c3 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
new file mode 100644
index 0000000..32c72ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int64/divide_int64.2.query.aql
@@ -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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c4 div $c1,"result2": $c4 div $c2,"result3": $c4 div $c3,"result4": $c4 div $c4,"result5": $c4 div $c5, "result6": $c4 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
new file mode 100644
index 0000000..5adb1b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/numeric/divide_int8/divide_int8.2.query.aql
@@ -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.
+ */
+
+let $c1 := int8("+1")
+let $c2 := int16("2")
+let $c3 := int32("+3")
+let $c4 := int64("-4")
+let $c5 := float("-5.5f")
+let $c6 := double("-6.5d")
+let $c8 := null
+return {"result1": $c1 div $c1,"result2": $c1 div $c2,"result3": $c1 div $c3,"result4": $c1 div $c4,"result5": $c1 div $c5, "result6": $c1 div $c6, "result7": $c6 div $c8, "result8": $c6 div [1][1]}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index bc9e7f7..0ab61e8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -36,11 +36,11 @@
let $str8:=substring($str7,0)
let $str9:="This is a test string"
-let $str10:=substring($str9,-1)
+let $str10:=substring($str9,-6)
let $str11:="This is a test string"
let $str12:="This is a another test string"
let $str13:=substring(string-concat([$str11,$str12]),20)
-let $str14:=substring("UC Irvine",string-length("UC Irvine")/2 - 1)
+let $str14:=substring("UC Irvine",string-length("UC Irvine") div 2 - 1)
return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
index 075af82..ecfcd94 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
@@ -18,5 +18,5 @@
*/
use dataverse test;
-let $c1 := substring("HEllow",-1)
+let $c1 := substring("HEllow",-3)
return {"result1": $c1}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.query.sqlpp
new file mode 100644
index 0000000..c11dd76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.1.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.
+ */
+
+{
+ "t1": missingif(missing, missing) is missing,
+ "t2": missingif(missing, 2) is missing,
+ "t3": missingif(2, missing) is missing,
+ "t4": missingif(missing, null) is missing,
+ "t5": missingif(null, missing) is missing,
+ "t6": missingif(null, null) is null,
+ "t7": missingif(null, 2) is null,
+ "t8": missingif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.2.query.sqlpp
new file mode 100644
index 0000000..b5f08ec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/missingif/missingif.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.
+ */
+
+from
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_missing(missing_if(t1[1], t2[1]), "MISSING") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.query.sqlpp
new file mode 100644
index 0000000..44e64d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.1.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.
+ */
+
+{
+ "t1": nanif(missing, missing) is missing,
+ "t2": nanif(missing, 2) is missing,
+ "t3": nanif(2, missing) is missing,
+ "t4": nanif(missing, null) is missing,
+ "t5": nanif(null, missing) is missing,
+ "t6": nanif(null, null) is null,
+ "t7": nanif(null, 2) is null,
+ "t8": nanif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.2.query.sqlpp
new file mode 100644
index 0000000..05e003e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nanif/nanif.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.
+ */
+
+from
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(nan_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.query.sqlpp
new file mode 100644
index 0000000..abd4a4b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.1.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.
+ */
+
+{
+ "t1": neginfif(missing, missing) is missing,
+ "t2": neginfif(missing, 2) is missing,
+ "t3": neginfif(2, missing) is missing,
+ "t4": neginfif(missing, null) is missing,
+ "t5": neginfif(null, missing) is missing,
+ "t6": neginfif(null, null) is null,
+ "t7": neginfif(null, 2) is null,
+ "t8": neginfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.2.query.sqlpp
new file mode 100644
index 0000000..67dcd34
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/neginfif/neginfif.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.
+ */
+
+from
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(neginf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.query.sqlpp
new file mode 100644
index 0000000..69ef8e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.1.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.
+ */
+
+{
+ "t1": nullif(missing, missing) is missing,
+ "t2": nullif(missing, 2) is missing,
+ "t3": nullif(2, missing) is missing,
+ "t4": nullif(missing, null) is missing,
+ "t5": nullif(null, missing) is missing,
+ "t6": nullif(null, null) is null,
+ "t7": nullif(null, 2) is null,
+ "t8": nullif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.2.query.sqlpp
new file mode 100644
index 0000000..d5d6be3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/nullif/nullif.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.
+ */
+
+from
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], if_null(null_if(t1[1], t2[1]), "NULL") ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.query.sqlpp
new file mode 100644
index 0000000..399f4d4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.1.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.
+ */
+
+{
+ "t1": posinfif(missing, missing) is missing,
+ "t2": posinfif(missing, 2) is missing,
+ "t3": posinfif(2, missing) is missing,
+ "t4": posinfif(missing, null) is missing,
+ "t5": posinfif(null, missing) is missing,
+ "t6": posinfif(null, null) is null,
+ "t7": posinfif(null, 2) is null,
+ "t8": posinfif(2, null) is null
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.2.query.sqlpp
new file mode 100644
index 0000000..cf25c1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/comparison/posinfif/posinfif.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.
+ */
+
+from
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t1,
+ [ [0, true], [1, false], [2,2], [3,3], [4,"abc"], [5,"def"] ] t2
+select value [ t1[1], t2[1], to_string(posinf_if(t1[1], t2[1])) ]
+order by t1[0], t2[0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.10.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
index a9ed09d..0c56f0a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.6.query.sqlpp
@@ -27,6 +27,4 @@
use experiments;
-select value count(*) from DBLP0;
-select value count(*) from DBLP1;
-select value count(*) from DBLP2;
\ No newline at end of file
+select value count(*) from DBLP0;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
new file mode 100644
index 0000000..62b920b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.query.sqlpp
@@ -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.
+ */
+/*
+ * Description : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
new file mode 100644
index 0000000..b7f38b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.8.query.sqlpp
@@ -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.
+ */
+/*
+ * Description : Create a socket feed with datatype without auuid and connect
+ * it to three different datasets with different datatype: one with auuid, one
+ * open datatype with auuid and one without auuid using one of the attribute
+ * as primary key.
+ * Expected Res : Success
+ * Date : 20th Mar 2017
+ */
+
+use experiments;
+
+select value count(*) from DBLP2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.7.server.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/push-socket-with-auuid/push-socket-with-auuid.9.server.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.1.ddl.sqlpp
new file mode 100644
index 0000000..95c1c38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.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 : Test filters with autogenerated keys
+ * Expected Res : Success
+ * Date : 20th Mar 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type UserLocation as closed {
+recordId: uuid,
+location: circle,
+userName: string
+};
+
+
+create dataset UserLocations(UserLocation)
+primary key recordId autogenerated with filter on userName;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.2.update.sqlpp
new file mode 100644
index 0000000..cee10ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.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 test;
+
+insert into UserLocations(
+{"userName" : "c1121u1" , "location" : circle("4171.58,1083.41 100.0")}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.3.query.sqlpp
new file mode 100644
index 0000000..945fc1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/filters/filter-auto-key/filter-auto-key.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.
+ */
+
+use test;
+
+select l.userName from UserLocations l
+where l.userName = "c1121u1";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
new file mode 100644
index 0000000..24f4ed6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/GeoJSONQueries.xml
@@ -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.
+ !-->
+<test-group name="geojson">
+ <test-case FilePath="geojson">
+ <compilation-unit name="datatype">
+ <output-dir compare="Text">datatype</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="geojson">
+ <compilation-unit name="single-method">
+ <output-dir compare="Text">single-method</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="geojson">
+ <compilation-unit name="two-geometries">
+ <output-dir compare="Text">two-geometries</output-dir>
+ </compilation-unit>
+ </test-case>
+</test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
new file mode 100644
index 0000000..61a7be6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/analysis.12.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT VALUE {"PointN": st_point_n(geo.myGeometry,1), "StartPoint":st_start_point(geo.myGeometry),"Envelope":st_envelope(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
new file mode 100644
index 0000000..dd378a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.1.ddl.sqlpp
@@ -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.
+ */
+
+drop dataverse GeoJSON if exists;
+create dataverse GeoJSON;
+
+use GeoJSON;
+
+CREATE TYPE GeometryType AS{
+ id : int,
+ myGeometry : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.sqlpp
new file mode 100644
index 0000000..3ab3bc6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.2.update.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.
+ */
+use GeoJSON;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry": st_geom_from_geojson({"type":"Point","coordinates":[-118.4,33.93]})},
+{"id": 124, "myGeometry": st_geom_from_geojson({"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]]})},
+{"id": 126, "myGeometry": st_geom_from_geojson({"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]]})},
+{"id": 127, "myGeometry": st_geom_from_geojson({"type": "MultiPoint","coordinates": [[10, 40], [40, 30], [20, 20], [30, 10]]})},
+{"id": 128, "myGeometry": st_geom_from_geojson({"type": "MultiLineString","coordinates": [[[10, 10], [20, 20], [10, 40]],[[40, 40], [30, 30], [40, 20], [30, 10]]]})},
+{"id": 129, "myGeometry": st_geom_from_geojson({"type": "MultiPolygon","coordinates": [[[[40, 40], [20, 45], [45, 30], [40, 40]]],[[[20, 35], [10, 30], [10, 10], [30, 5], [45, 20], [20, 35]],[[30, 20], [20, 15], [20, 25], [30, 20]]]]})},
+{"id": 130, "myGeometry": st_make_point(-71.1043443253471, 42.3150676015829)},
+{"id": 131, "myGeometry": st_make_point(1.0,2.0,3.0)},
+{"id": 132, "myGeometry": st_make_point(1.0,2.0,3.0,4.0)},
+{"id": 133, "myGeometry": st_geom_from_text('POLYGON((743238 2967416,743238 2967450,743265 2967450,743265.625 2967416,743238 2967416))')},
+{"id": 134, "myGeometry": st_geom_from_wkb(hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"))},
+{"id": 135, "myGeometry": st_line_from_multipoint(st_geom_from_text('MULTIPOINT(1 2 , 4 5 , 7 8 )'))},
+{"id": 136, "myGeometry": st_make_envelope(10, 10, 11, 11, 4326)},
+{"id": 137, "myGeometry": st_geom_from_text("POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10),(20 30, 35 35, 30 20, 20 30))")}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
new file mode 100644
index 0000000..540bce1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/datatype_definition.3.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT * FROM Geometries ORDER BY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
new file mode 100644
index 0000000..beb8879
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.17.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT VALUE {"IsClosed":st_is_closed(geo.myGeometry), "IsCollection":st_is_collection(geo.myGeometry),"IsEmpty":st_is_empty(geo.myGeometry), "IsSimple":st_is_simple(geo.myGeometry)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.4.query.sqlpp
new file mode 100644
index 0000000..4a19b47
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.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.
+ */
+use GeoJSON;
+
+SELECT VALUE {"Type": geometry_type(geo.myGeometry), "Area": st_area(geo.myGeometry),
+"Coordinate dimension": st_coord_dim(geo.myGeometry), "Dimension":st_dimension(geo.myGeometry),
+"NPoints":st_n_points(geo.myGeometry), "XMax":st_x_max(geo.myGeometry),"XMin":st_x_min(geo.myGeometry),
+"YMax":st_y_max(geo.myGeometry), "YMin":st_y_min(geo.myGeometry), "Binary": st_as_binary(geo.myGeometry),
+"GeoJSON":st_as_geojson(geo.myGeometry),"WKT":st_as_text(geo.myGeometry)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
new file mode 100644
index 0000000..4868d4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.5.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT VALUE {"X":st_x(geo.myGeometry),"Y":st_y(geo.myGeometry),"Z":st_z(geo.myGeometry), "M":st_m(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Point" ;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
new file mode 100644
index 0000000..d7e32a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.6.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT VALUE {"NumInteriorRings":st_num_interior_rings(geo.myGeometry), "ExteriorRing":st_exterior_ring(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="Polygon";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
new file mode 100644
index 0000000..9fb187d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/datatype/primitive.9.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 GeoJSON;
+
+SELECT VALUE {"Length": st_length(geo.myGeometry), "Boundary":st_boundary(geo.myGeometry)} FROM Geometries geo WHERE geometry_type(geo.myGeometry)="LineString" OR geometry_type(geo.myGeometry)="MultiLineString" ORDER BY geo.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
new file mode 100644
index 0000000..01c2563
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/analysis.10.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+st_geometry_n(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'),2);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.query.sqlpp
new file mode 100644
index 0000000..28ee45f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.13.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.
+ */
+
+st_is_collection(st_polygonize([st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")]));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.query.sqlpp
new file mode 100644
index 0000000..54b4450
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/predicate.14.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.
+ */
+
+st_is_empty(st_geom_from_text("LINESTRING EMPTY"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
new file mode 100644
index 0000000..9924d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/single-method/primitive.7.query.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+st_num_geometries(st_geom_from_text('GEOMETRYCOLLECTION(MULTIPOINT(-2 3 , -2 2),LINESTRING(5 5 ,10 10),POLYGON((-7 4.2,-7.1 5,-7.1 4.3,-7 4.2)))'));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.query.sqlpp
new file mode 100644
index 0000000..3a889b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/analysis.16.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.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Union": st_union(geo.myGeometry1, geo.myGeometry2),"Intersection":st_intersection(geo.myGeometry1,
+geo.myGeometry2),"SymDifference":st_sym_difference(geo.myGeometry1,geo.myGeometry2)} FROM Geometries geo;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.sqlpp
new file mode 100644
index 0000000..7084fe3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.11.update.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.
+ */
+use GeoJSON1;
+
+INSERT INTO Geometries ([
+{"id": 123, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 124, "myGeometry1": st_geom_from_text("LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)"), "myGeometry2": st_geom_from_text("LINESTRING(0 2,1 2,2 2,3 3,4 2,5 2)")},
+{"id": 126, "myGeometry1": st_geom_from_text('POINT(0 0)'), "myGeometry2": st_geom_from_text('LINESTRING ( 0 0, 0 2 )')},
+{"id": 127, "myGeometry1": st_geom_from_text('LINESTRING(1 1,-1 -1,2 3.5,1 3,1 2,2 1)'), "myGeometry2": st_make_point(-1, -1)},
+{"id": 128, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('LINESTRING(0 2,1 2,2 2,3 2,4 2,5 2)')},
+{"id": 129, "myGeometry1": st_geom_from_text('LINESTRING(1 1,2 2,3 3,4 4, 5 5,6 6)'), "myGeometry2": st_geom_from_text('POINT(0 0)')},
+{"id": 130, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 10 10)'), "myGeometry2": st_geom_from_text('LINESTRING(0 0, 5 5, 10 10)')},
+{"id": 131, "myGeometry1": st_geom_from_text('LINESTRING(1 2, 3 4)'), "myGeometry2": st_geom_from_text('LINESTRING(5 6, 7 8)')},
+{"id": 132, "myGeometry1": st_geom_from_text('LINESTRING(0 0, 1 1, 0 2)'), "myGeometry2": st_geom_from_text('POINT(0 2)')},
+{"id": 133, "myGeometry1": st_geom_from_text('POLYGON((0.25 0.25, 0.5 0.25, 0.5 0.5, 0.25 0.5, 0.25 0.25 ))'), "myGeometry2": st_geom_from_text('POLYGON ((0 0, 1 0, 1 1, 0 1, 0 0))')}
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.ddl.sqlpp
new file mode 100644
index 0000000..671d151
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/datatype_definition.8.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.
+ */
+
+drop dataverse GeoJSON1 if exists;
+create dataverse GeoJSON1;
+
+use GeoJSON1;
+
+CREATE TYPE GeometryType AS{
+ id : int,
+ myGeometry1 : geometry,
+ myGeometry2 : geometry
+};
+
+CREATE DATASET Geometries (GeometryType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.15.query.sqlpp
new file mode 100644
index 0000000..74a3fe2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/geojson/two-geometries/primitive.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.
+ */
+use GeoJSON1;
+
+SELECT VALUE {"Distance":st_distance(geo.myGeometry1, geo.myGeometry2), "Intersects":st_intersects(geo.myGeometry1,
+geo.myGeometry2), "Contains":st_contains(geo.myGeometry1, geo.myGeometry2), "Crosses":st_crosses(geo.myGeometry1,
+geo.myGeometry2), "Disjoint":st_disjoint(geo.myGeometry1, geo.myGeometry2), "Equals":st_equals(geo.myGeometry1,
+geo.myGeometry2), "Overlaps":st_overlaps(geo.myGeometry1, geo.myGeometry2), "Relate":st_relate(geo.myGeometry1,
+geo.myGeometry2, "FF1FF0102"), "Touches":st_touches(geo.myGeometry1, geo.myGeometry2), "Within":st_within(
+geo.myGeometry1, geo.myGeometry2)} FROM Geometries geo;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
new file mode 100644
index 0000000..e2782c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description : Test push down limit and select condition into the primary index lookup operator
+ * Expected Result : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+ l_orderkey : bigint,
+ l_partkey : bigint,
+ l_suppkey : bigint,
+ l_linenumber : bigint,
+ l_quantity : bigint,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create index idx_LineItem_suppkey on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.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 test;
+
+
+load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.query.sqlpp
new file mode 100644
index 0000000..b8eac5d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.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.
+ */
+
+use test;
+
+explain select element c
+from LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.query.sqlpp
new file mode 100644
index 0000000..212479a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.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.
+ */
+
+use test;
+
+
+select element c
+from LineItem as c
+where c.l_suppkey < 150 AND l_extendedprice < 10000
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
new file mode 100644
index 0000000..e9daaad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description : Test push down limit into primary key lookup operator
+ * Expected Result : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.LineItemType as
+ closed {
+ l_orderkey : bigint,
+ l_partkey : bigint,
+ l_suppkey : bigint,
+ l_linenumber : bigint,
+ l_quantity : bigint,
+ l_extendedprice : double,
+ l_discount : double,
+ l_tax : double,
+ l_returnflag : string,
+ l_linestatus : string,
+ l_shipdate : string,
+ l_commitdate : string,
+ l_receiptdate : string,
+ l_shipinstruct : string,
+ l_shipmode : string,
+ l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create index idx_LineItem_suppkey on LineItem (l_suppkey) type btree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.2.update.sqlpp
new file mode 100644
index 0000000..546a831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.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 test;
+
+
+load dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.query.sqlpp
new file mode 100644
index 0000000..43f7d94
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.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.
+ */
+
+use test;
+
+explain select element c
+from LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.query.sqlpp
new file mode 100644
index 0000000..026aed7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.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.
+ */
+
+use test;
+
+
+select element c
+from LineItem as c
+where (c.l_suppkey < 150)
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
new file mode 100644
index 0000000..f214dc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.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.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description : Test push down limit and select condition into primary index scan operator
+ * Expected Result : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+ id : bigint,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
new file mode 100644
index 0000000..2e0f056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.query.sqlpp
new file mode 100644
index 0000000..657bfb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+ select element paper
+ from DBLP1 as paper
+ where contains(dblpid, 'kimL89')
+ limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.query.sqlpp
new file mode 100644
index 0000000..6459b5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from DBLP1 as paper
+where contains(dblpid, 'kimL89')
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
new file mode 100644
index 0000000..162cc35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.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.
+ */
+/* scan and print a delimited text file */
+
+/*
+ * Description : Test push down limit into the primary index scan operator
+ * Expected Result : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+ id : bigint,
+ dblpid : string,
+ title : string,
+ authors : string,
+ misc : string
+};
+
+create dataset DBLP1(DBLPType) primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
new file mode 100644
index 0000000..2e0f056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+
+load dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.query.sqlpp
new file mode 100644
index 0000000..d2cb38a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+ select element paper
+ from DBLP1 as paper
+ limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.query.sqlpp
new file mode 100644
index 0000000..5b7c5be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+select element paper
+from DBLP1 as paper
+limit 5 offset 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
new file mode 100644
index 0000000..c4d77ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.1.ddl.sqlpp
@@ -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.
+ */
+
+/*
+ * Description : Processed objects metrics on secondary index scan when it's an index-only plan
+ * Expected Res : Success
+ * Date : 27 Apr 2018
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.AddressType as
+{
+ number : bigint,
+ street : string,
+ city : string
+};
+
+create type test.CustomerType as
+ closed {
+ cid : bigint,
+ name : string,
+ age : bigint?,
+ address : AddressType?,
+ lastorder : {
+ oid : bigint,
+ total : float
+ }
+};
+
+create dataset Customers(CustomerType) primary key cid;
+create index customer_name_idx on Customers(name);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.2.update.sqlpp
new file mode 100644
index 0000000..6a5fd9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.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 test;
+
+load dataset Customers using localfs
+ ((`path`=`asterix_nc1://data/custord-tiny/customer-tiny-neg.adm`),
+ (`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.sqlpp
new file mode 100644
index 0000000..881ed24
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.3.metrics.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 test;
+
+select count(*) from Customers
+where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.sqlpp
new file mode 100644
index 0000000..f12a2b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index-index-only/secondary-index-index-only.4.ddl.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.
+ */
+
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
index 1c5126f..95d4681 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/secondary-index/secondary-index.3.metrics.sqlpp
@@ -24,5 +24,7 @@
use test;
+set noindexonly 'true';
+
select count(*) from Customers
where name = "Marvella Loud";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.query.sqlpp
new file mode 100644
index 0000000..60229d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_08/case_08.1.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
+ CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" as v1,
+ ( CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END ) LIKE "a%" as v2,
+ [ CASE WHEN get_year(current_date()) > 0 THEN "abc" ELSE "def" END LIKE "a%" ] as v3
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.sqlpp
new file mode 100644
index 0000000..bf35f87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.1.query.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.
+ */
+
+{
+ "t1": array_length((
+ from range(1, 6) t
+ select distinct value random()
+ )),
+
+ "t2": array_length((
+ from range(1, 6) t
+ select distinct value random(t)
+ )),
+
+ "t3": array_length((
+ from range(1, 6) t
+ select distinct value random(unix_time_from_datetime_in_ms(current_datetime()))
+ )),
+
+ "t4": array_length((
+ from [ tinyint("1"), smallint("2"), integer("3"), bigint("4"), float("5"), double("6") ] t
+ select distinct value random(t)
+ )),
+
+ "t5": [ random(missing) is missing, random(null) is null ]
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.sqlpp
new file mode 100644
index 0000000..f650e0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.1.ddl.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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+ id : bigint,
+ point : point,
+ kwds : string,
+ line1 : line,
+ line2 : line,
+ poly1 : polygon,
+ poly2 : polygon,
+ rec : rectangle
+};
+
+create type test.MyRecord as
+{
+ pid: uuid,
+ nested : MyRecordtmp?
+};
+
+create dataset MyDatatmp(MyRecordtmp) primary key id;
+
+create dataset MyData(MyRecord) primary key pid autogenerated;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.sqlpp
new file mode 100644
index 0000000..8199dbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.2.update.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.
+ */
+
+use test;
+
+
+load dataset MyDatatmp using localfs ((`path`=`asterix_nc1://data/spatial/spatialData.json`),(`format`=`adm`));
+
+insert into MyData
+select element {'nested':c}
+from MyDatatmp as c
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.3.ddl.sqlpp
new file mode 100644
index 0000000..5683fe0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.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 test;
+
+
+create index rtree_index_point on MyData (nested.point) type rtree;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.4.query.sqlpp
new file mode 100644
index 0000000..93665a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nested-index/index-selection/rtree-secondary-index-optional/rtree-secondary-index-optional.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.
+ */
+
+use test;
+
+
+select element {'id':o.nested.id}
+from MyData as o
+where test.`spatial-intersect`(o.nested.point,test.`create-polygon`([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+order by o.nested.id
+;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.1.ddl.sqlpp
new file mode 100644
index 0000000..c4af7d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/nestrecords/nested-optional-pk/nested-optional-pk.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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.MyRecordtmp as
+{
+ id : bigint,
+ point : point
+};
+
+create type test.MyRecord as
+{
+ nested : MyRecordtmp?
+};
+
+create dataset MyData(MyRecord) primary key nested.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
index 81f2a3f..77e8d91 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/is/is.1.query.sqlpp
@@ -41,5 +41,11 @@
"21": 1 is valued,
"22": null is not valued,
"23": missing is not valued,
- "24": 1 is not valued
+ "24": 1 is not valued,
+ "25": null is known,
+ "26": missing is known,
+ "27": 1 is known,
+ "28": null is not known,
+ "29": missing is not known,
+ "30": 1 is not known
};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.query.sqlpp
new file mode 100644
index 0000000..fe67901
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/const/const.1.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.
+ */
+
+{
+ "e": trunc(e(), 2),
+ "pi": trunc(pi(), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.query.sqlpp
new file mode 100644
index 0000000..80e002e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/degrees/degrees.1.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.
+ */
+
+{
+ "t1": tobigint(degrees(pi()))
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.query.sqlpp
new file mode 100644
index 0000000..cca3dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int16/divide_int16.2.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.
+ */
+
+{'result1':(smallint('2') div tinyint('+1')),'result2':(smallint('2') div smallint('2')),'result3':(smallint('2') div integer('+3')),'result4':(smallint('2') div bigint('-4')),'result5':(smallint('2') div float('-5.5f')),'result6':(smallint('2') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
new file mode 100644
index 0000000..7ce0f9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int32/divide_int32.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+{'result1':(integer('+3') div tinyint('+1')),'result2':(integer('+3') div smallint('2')),'result3':(integer('+3') div integer('+3')),'result4':(integer('+3') div bigint('-4')),'result5':(integer('+3') div float('-5.5f')),'result6':(integer('+3') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.query.sqlpp
new file mode 100644
index 0000000..2f850c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int64/divide_int64.2.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.
+ */
+
+{'result1':(bigint('-4') div tinyint('+1')),'result2':(bigint('-4') div smallint('2')),'result3':(bigint('-4') div integer('+3')),'result4':(bigint('-4') div bigint('-4')),'result5':(bigint('-4') div float('-5.5f')),'result6':(bigint('-4') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
new file mode 100644
index 0000000..5cf4b1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/divide_int8/divide_int8.2.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+{'result1':(tinyint('+1') div tinyint('+1')),'result2':(tinyint('+1') div smallint('2')),'result3':(tinyint('+1') div integer('+3')),'result4':(tinyint('+1') div bigint('-4')),'result5':(tinyint('+1') div float('-5.5f')),'result6':(tinyint('+1') div double('-6.5d')),'result7':(double('-6.5d') div null), 'result8':double('-6.5d') div {}.a};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.query.sqlpp
new file mode 100644
index 0000000..6d04878
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/radians/radians.1.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.
+ */
+
+{
+ "t1": trunc(radians(180), 2)
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.query.sqlpp
new file mode 100644
index 0000000..e127372
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/reverse/reverse.1.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.
+ */
+
+ {
+ "t1": [ reverse(missing) is missing, reverse(null) is null ],
+ "t2": reverse(""),
+ "t3": reverse("abcd"),
+ "t4": string_to_codepoint(reverse("a\u00D7\u2103\u00F7\u2109b")),
+ "t5": ( from ["ab", "abc", "abcd"] t select value reverse(t) order by t )
+ }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
index b9c79d0..d39e51e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr01/substr01.3.query.sqlpp
@@ -23,4 +23,4 @@
* Date : 18th April 2012
*/
-{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-1),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') / 2 - 1))};
+{'str2':substring('Hello World',9),'str4':substring('This is a test string',20),'str6':substring('This is a test string',21),'str8':substring('This is a test string',0),'str10':substring('This is a test string',-6),'str13':substring(`string-concat`(['This is a test string','This is a another test string']),20),'str14':substring('UC Irvine',(`string-length`('UC Irvine') div 2 - 1))};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
index 8ea572c..ae943d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substring2-4/substring2-4.3.query.sqlpp
@@ -20,4 +20,4 @@
use test;
-{'result1':test.substring('HEllow',-1)};
+{'result1':test.substring('HEllow',-3)};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
index 1380ed2..efe17a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.3.query.sqlpp
@@ -17,8 +17,4 @@
* under the License.
*/
use test;
-
-
select element test.computeBonus(-1,-1);
-select element test.computeBonus(1,-1);
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.4.query.sqlpp
new file mode 100644
index 0000000..5ee25ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/udf25/udf25.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.
+ */
+use test;
+select element test.computeBonus(1,-1);
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 0e619cc..cc4c573 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
@@ -37,7 +37,7 @@
"txn\.lock\.timeout\.sweepthreshold" : 10000,
"txn\.lock\.timeout\.waitthreshold" : 60000,
"txn\.log\.buffer\.numpages" : 8,
- "txn\.log\.buffer\.pagesize" : 131072,
+ "txn\.log\.buffer\.pagesize" : 4194304,
"txn\.log\.checkpoint\.history" : 0,
"txn\.log\.checkpoint\.lsnthreshold" : 67108864,
"txn\.log\.checkpoint\.pollfrequency" : 120,
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 b3a3eed..dd7dfb7 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
@@ -37,7 +37,7 @@
"txn\.lock\.timeout\.sweepthreshold" : 10000,
"txn\.lock\.timeout\.waitthreshold" : 60000,
"txn\.log\.buffer\.numpages" : 8,
- "txn\.log\.buffer\.pagesize" : 131072,
+ "txn\.log\.buffer\.pagesize" : 4194304,
"txn\.log\.checkpoint\.history" : 0,
"txn\.log\.checkpoint\.lsnthreshold" : 67108864,
"txn\.log\.checkpoint\.pollfrequency" : 120,
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 75e879c..71200c1 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
@@ -37,7 +37,7 @@
"txn\.lock\.timeout\.sweepthreshold" : 10000,
"txn\.lock\.timeout\.waitthreshold" : 60000,
"txn\.log\.buffer\.numpages" : 8,
- "txn\.log\.buffer\.pagesize" : 131072,
+ "txn\.log\.buffer\.pagesize" : 4194304,
"txn\.log\.checkpoint\.history" : 0,
"txn\.log\.checkpoint\.lsnthreshold" : 67108864,
"txn\.log\.checkpoint\.pollfrequency" : 120,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
new file mode 100644
index 0000000..fca4cbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/missingif/missingif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "MISSING" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "MISSING" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "MISSING" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "MISSING" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "MISSING" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "MISSING" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
new file mode 100644
index 0000000..cfcc525
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nanif/nanif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NaN" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "NaN" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "NaN" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "NaN" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NaN" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NaN" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
new file mode 100644
index 0000000..1b75b97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/neginfif/neginfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "-INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "-INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "-INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "-INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "-INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "-INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
new file mode 100644
index 0000000..5c53877
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/nullif/nullif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "NULL" ]
+[ true, false, true ]
+[ true, 2, true ]
+[ true, 3, true ]
+[ true, "abc", true ]
+[ true, "def", true ]
+[ false, true, false ]
+[ false, false, "NULL" ]
+[ false, 2, false ]
+[ false, 3, false ]
+[ false, "abc", false ]
+[ false, "def", false ]
+[ 2, true, 2 ]
+[ 2, false, 2 ]
+[ 2, 2, "NULL" ]
+[ 2, 3, 2 ]
+[ 2, "abc", 2 ]
+[ 2, "def", 2 ]
+[ 3, true, 3 ]
+[ 3, false, 3 ]
+[ 3, 2, 3 ]
+[ 3, 3, "NULL" ]
+[ 3, "abc", 3 ]
+[ 3, "def", 3 ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "NULL" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "NULL" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
new file mode 100644
index 0000000..b0a2b5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.1.adm
@@ -0,0 +1 @@
+{ "t1": true, "t2": true, "t3": true, "t4": true, "t5": true, "t6": true, "t7": true, "t8": true }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
new file mode 100644
index 0000000..ebc67b3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/comparison/posinfif/posinfif.2.adm
@@ -0,0 +1,36 @@
+[ true, true, "INF" ]
+[ true, false, "true" ]
+[ true, 2, "true" ]
+[ true, 3, "true" ]
+[ true, "abc", "true" ]
+[ true, "def", "true" ]
+[ false, true, "false" ]
+[ false, false, "INF" ]
+[ false, 2, "false" ]
+[ false, 3, "false" ]
+[ false, "abc", "false" ]
+[ false, "def", "false" ]
+[ 2, true, "2" ]
+[ 2, false, "2" ]
+[ 2, 2, "INF" ]
+[ 2, 3, "2" ]
+[ 2, "abc", "2" ]
+[ 2, "def", "2" ]
+[ 3, true, "3" ]
+[ 3, false, "3" ]
+[ 3, 2, "3" ]
+[ 3, 3, "INF" ]
+[ 3, "abc", "3" ]
+[ 3, "def", "3" ]
+[ "abc", true, "abc" ]
+[ "abc", false, "abc" ]
+[ "abc", 2, "abc" ]
+[ "abc", 3, "abc" ]
+[ "abc", "abc", "INF" ]
+[ "abc", "def", "abc" ]
+[ "def", true, "def" ]
+[ "def", false, "def" ]
+[ "def", 2, "def" ]
+[ "def", 3, "def" ]
+[ "def", "abc", "def" ]
+[ "def", "def", "INF" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
index 58d9bdc..dd41cbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_04/customer_q_04.1.adm
@@ -1,3 +1,3 @@
-{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20, "-cashBack": -100 } }
-{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70, "-cashBack": -350 } }
-{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90, "-cashBack": -450 } }
+{ "id": 775, "custname": "Jodi Rotruck", "age": null, "MathcashBack": { "cashBack": 100, "cashBack+5": 105, "cashBack-5": 95, "cashBack*5": 500, "cashBack/5": 20.0, "-cashBack": -100 } }
+{ "id": 5, "custname": "Jodi Alex", "age": 19, "MathcashBack": { "cashBack": 350, "cashBack+5": 355, "cashBack-5": 345, "cashBack*5": 1750, "cashBack/5": 70.0, "-cashBack": -350 } }
+{ "id": 4, "custname": "Mary Carey", "age": 12, "MathcashBack": { "cashBack": 450, "cashBack+5": 455, "cashBack-5": 445, "cashBack*5": 2250, "cashBack/5": 90.0, "-cashBack": -450 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
index 89f8083..2990e0e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/custord/customer_q_05/customer_q_05.1.adm
@@ -1,3 +1,3 @@
{ "custname": "Jodi Rotruck", "age": null, "MathAge": { "age": null, "age+5": null, "age-5": null, "age*5": null, "age/5": null, "-age": null } }
-{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3, "-age": -19 } }
-{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2, "-age": -12 } }
+{ "custname": "Jodi Alex", "age": 19, "MathAge": { "age": 19, "age+5": 24, "age-5": 14, "age*5": 95, "age/5": 3.8, "-age": -19 } }
+{ "custname": "Mary Carey", "age": 12, "MathAge": { "age": 12, "age+5": 17, "age-5": 7, "age*5": 60, "age/5": 2.4, "-age": -12 } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
index 51fdf04..f599e28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.1.adm
@@ -1,3 +1 @@
10
-10
-10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.2.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
new file mode 100644
index 0000000..f599e28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/push-socket-with-auuid/push-socket-with-auuid.3.adm
@@ -0,0 +1 @@
+10
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
new file mode 100644
index 0000000..022f6a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/filters/filter-auto-key/filter-auto-key.1.adm
@@ -0,0 +1 @@
+{ "userName": "c1121u1" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
new file mode 100644
index 0000000..2a16ab1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/datatype.3.adm
@@ -0,0 +1,14 @@
+{ "Geometries": { "id": 123, "myGeometry": {"type":"Point","coordinates":[-118.4,33.93],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 124, "myGeometry": {"type":"Polygon","coordinates":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 126, "myGeometry": {"type":"LineString","coordinates":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 127, "myGeometry": {"type":"MultiPoint","coordinates":[[10,40],[40,30],[20,20],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 128, "myGeometry": {"type":"MultiLineString","coordinates":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],"crs":null} } }
+{ "Geometries": { "id": 129, "myGeometry": {"type":"MultiPolygon","coordinates":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],"crs":null} } }
+{ "Geometries": { "id": 130, "myGeometry": {"type":"Point","coordinates":[-71.1043443253471,42.3150676015829],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 131, "myGeometry": {"type":"Point","coordinates":[1,2,3],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 132, "myGeometry": {"type":"Point","coordinates":[1,2,3,4],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 133, "myGeometry": {"type":"Polygon","coordinates":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 134, "myGeometry": {"type":"LineString","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 135, "myGeometry": {"type":"LineString","coordinates":[[1,2],[4,5],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 136, "myGeometry": {"type":"Polygon","coordinates":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
+{ "Geometries": { "id": 137, "myGeometry": {"type":"Polygon","coordinates":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
new file mode 100644
index 0000000..b9029e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.12.adm
@@ -0,0 +1,3 @@
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-69.1991349,-12.6006222],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-69.199136,-12.6010968],[-69.1972972,-12.6010968],[-69.1972972,-12.5998133],[-69.199136,-12.5998133],[-69.199136,-12.6010968]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-69.199136,-12.599842],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[1,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[1,2],[7,2],[7,8],[1,8],[1,2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "PointN": {"type":"Point","coordinates":[-113.981,39.195],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "StartPoint": {"type":"Point","coordinates":[-113.98,39.198],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Envelope": {"type":"Polygon","coordinates":[[[-113.981,39.195],[-113.98,39.195],[-113.98,39.198],[-113.981,39.198],[-113.981,39.195]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
new file mode 100644
index 0000000..048a6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.17.adm
@@ -0,0 +1,14 @@
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": true, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": false, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
+{ "IsClosed": true, "IsCollection": false, "IsEmpty": false, "IsSimple": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
new file mode 100644
index 0000000..3af2594
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.4.adm
@@ -0,0 +1,14 @@
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 13, "XMax": -69.1972972, "XMin": -69.199136, "YMax": -12.5998133, "YMin": -12.6010968, "Binary": hex("01020000000D00000081BF4EA0BE4C51C0CE80C4C0843329C033DDEBA4BE4C51C0D388997D1E3329C064D7ACE9B04C51C05787927F1C3329C02347DF49B04C51C0B9C49107223329C0F4DCE79DAF4C51C054B76922223329C0184339D1AE4C51C042A89C40223329C048E17A14AE4C51C010DBCCD71B3329C007DB3E9AA04C51C0636996BA1A3329C0E9656F84A04C51C05ADD45F35B3329C04E9B711AA24C51C033373A42613329C06CE22E0CA24C51C066D24B31AC3329C05F454607A44C51C065170CAEB93329C07C8C03F9A34C51C0E475B3F5C23329C0"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-69.1991349,-12.6006222],[-69.199136,-12.599842],[-69.1982979,-12.5998268],[-69.1982598,-12.599869],[-69.1982188,-12.5998698],[-69.19817,-12.5998707],[-69.198125,-12.5998218],[-69.1973024,-12.5998133],[-69.1972972,-12.6003109],[-69.197394,-12.6003514],[-69.1973906,-12.6009231],[-69.1975115,-12.601026],[-69.1975081,-12.6010968]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-69.1991349 -12.6006222, -69.199136 -12.599842, -69.1982979 -12.5998268, -69.1982598 -12.599869, -69.1982188 -12.5998698, -69.19817 -12.5998707, -69.198125 -12.5998218, -69.1973024 -12.5998133, -69.1972972 -12.6003109, -69.197394 -12.6003514, -69.1973906 -12.6009231, -69.1975115 -12.601026, -69.1975081 -12.6010968)" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 3, "XMax": 7.0, "XMin": 1.0, "YMax": 8.0, "YMin": 2.0, "Binary": hex("010200000003000000000000000000F03F0000000000000040000000000000104000000000000014400000000000001C400000000000002040"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[1,2],[4,5],[7,8]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (1 2, 4 5, 7 8)" }
+{ "Type": "MultiPoint", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 4, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010400000004000000010100000000000000000024400000000000004440010100000000000000000044400000000000003E4001010000000000000000003440000000000000344001010000000000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiPoint\",\"coordinates\":[[10,40],[40,30],[20,20],[30,10]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "MULTIPOINT ((10 40), (40 30), (20 20), (30 10))" }
+{ "Type": "MultiPolygon", "Area": 712.5, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 11, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 5.0, "Binary": hex("01060000000200000001030000000100000004000000000000000000444000000000000044400000000000003440000000000080464000000000008046400000000000003E4000000000000044400000000000004440010300000002000000060000000000000000003440000000000080414000000000000024400000000000003E40000000000000244000000000000024400000000000003E4000000000000014400000000000804640000000000000344000000000000034400000000000804140040000000000000000003E40000000000000344000000000000034400000000000002E40000000000000344000000000000039400000000000003E400000000000003440"), "GeoJSON": "{\"type\":\"MultiPolygon\",\"coordinates\":[[[[40,40],[20,45],[45,30],[40,40]]],[[[20,35],[10,30],[10,10],[30,5],[45,20],[20,35]],[[30,20],[20,15],[20,25],[30,20]]]],\"crs\":null}", "WKT": "MULTIPOLYGON (((40 40, 20 45, 45 30, 40 40)), ((20 35, 10 30, 10 10, 30 5, 45 20, 20 35), (30 20, 20 15, 20 25, 30 20)))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -71.1043443253471, "XMin": -71.1043443253471, "YMax": 42.3150676015829, "YMin": 42.3150676015829, "Binary": hex("0101000000E538D293ADC651C0F3699A2254284540"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-71.1043443253471,42.3150676015829],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-71.1043443253471 42.3150676015829)" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 3, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01E9030000000000000000F03F00000000000000400000000000000840"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT Z (1 2 3)" }
+{ "Type": "Polygon", "Area": 1.3755215000294761E-8, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 6, "XMax": 8.760178, "XMin": 8.7599721, "YMax": 49.7103478, "YMin": 49.7102133, "Binary": hex("01030000000100000007000000B1BE26101B852140ED20C033EBDA4840C11DA8531E8521407694394CEADA4840BAF8DB9E20852140D5F89683EADA484098EF2AB5288521409557F844E8DA48400247020D3685214041F74086E9DA484014F131B32A8521408DE43CADECDA4840B1BE26101B852140ED20C033EBDA4840"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((8.7599721 49.7103028, 8.759997 49.7102752, 8.7600145 49.7102818, 8.7600762 49.7102133, 8.760178 49.7102516, 8.7600914 49.7103478, 8.7599721 49.7103028))" }
+{ "Type": "MultiLineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 7, "XMax": 40.0, "XMin": 10.0, "YMax": 40.0, "YMin": 10.0, "Binary": hex("010500000002000000010200000003000000000000000000244000000000000024400000000000003440000000000000344000000000000024400000000000004440010200000004000000000000000000444000000000000044400000000000003E400000000000003E40000000000000444000000000000034400000000000003E400000000000002440"), "GeoJSON": "{\"type\":\"MultiLineString\",\"coordinates\":[[[10,10],[20,20],[10,40]],[[40,40],[30,30],[40,20],[30,10]]],\"crs\":null}", "WKT": "MULTILINESTRING ((10 10, 20 20, 10 40), (40 40, 30 30, 40 20, 30 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 4, "Dimension": 0, "NPoints": 1, "XMax": 1.0, "XMin": 1.0, "YMax": 2.0, "YMin": 2.0, "Binary": hex("01B90B0000000000000000F03F000000000000004000000000000008400000000000001040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[1,2,3,4],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT ZM (1 2 3 4)" }
+{ "Type": "Polygon", "Area": 928.625, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 743265.625, "XMin": 743238.0, "YMax": 2967450.0, "YMin": 2967416.0, "Binary": hex("01030000000100000005000000000000008CAE264100000000BCA3464100000040C3AE264100000000BCA3464100000000C2AE264100000000CDA34641000000008CAE264100000000CDA34641000000008CAE264100000000BCA34641"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((743238 2967416, 743265.625 2967416, 743265 2967450, 743238 2967450, 743238 2967416))" }
+{ "Type": "LineString", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 1, "NPoints": 2, "XMax": -113.98, "XMin": -113.981, "YMax": 39.198, "YMin": 39.195, "Binary": hex("0102000000020000001F85EB51B87E5CC0D34D621058994340105839B4C87E5CC0295C8FC2F5984340"), "GeoJSON": "{\"type\":\"LineString\",\"coordinates\":[[-113.98,39.198],[-113.981,39.195]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "LINESTRING (-113.98 39.198, -113.981 39.195)" }
+{ "Type": "Polygon", "Area": 1.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 4, "XMax": 11.0, "XMin": 10.0, "YMax": 11.0, "YMin": 10.0, "Binary": hex("010300000001000000050000000000000000002440000000000000244000000000000026400000000000002440000000000000264000000000000026400000000000002440000000000000264000000000000024400000000000002440"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[10,10],[11,10],[11,11],[10,11],[10,10]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))" }
+{ "Type": "Point", "Area": 0.0, "Coordinate dimension": 2, "Dimension": 0, "NPoints": 1, "XMax": -118.4, "XMin": -118.4, "YMax": 33.93, "YMin": 33.93, "Binary": hex("01010000009A99999999995DC0D7A3703D0AF74040"), "GeoJSON": "{\"type\":\"Point\",\"coordinates\":[-118.4,33.93],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POINT (-118.4 33.93)" }
+{ "Type": "Polygon", "Area": 675.0, "Coordinate dimension": 2, "Dimension": 2, "NPoints": 7, "XMax": 45.0, "XMin": 10.0, "YMax": 45.0, "YMin": 10.0, "Binary": hex("0103000000020000000500000000000000008041400000000000002440000000000080464000000000008046400000000000002E40000000000000444000000000000024400000000000003440000000000080414000000000000024400400000000000000000034400000000000003E40000000000080414000000000008041400000000000003E40000000000000344000000000000034400000000000003E40"), "GeoJSON": "{\"type\":\"Polygon\",\"coordinates\":[[[35,10],[45,45],[15,40],[10,20],[35,10]],[[20,30],[35,35],[30,20],[20,30]]],\"crs\":{\"type\":\"name\",\"properties\":{\"name\":\"EPSG:4326\"}}}", "WKT": "POLYGON ((35 10, 45 45, 15 40, 10 20, 35 10), (20 30, 35 35, 30 20, 20 30))" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
new file mode 100644
index 0000000..950f68f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.5.adm
@@ -0,0 +1,4 @@
+{ "X": -71.1043443253471, "Y": 42.3150676015829, "Z": 0.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": NaN }
+{ "X": 1.0, "Y": 2.0, "Z": 3.0, "M": 4.0 }
+{ "X": -118.4, "Y": 33.93, "Z": 0.0, "M": NaN }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
new file mode 100644
index 0000000..044e31a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.6.adm
@@ -0,0 +1,4 @@
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[8.7599721,49.7103028],[8.759997,49.7102752],[8.7600145,49.7102818],[8.7600762,49.7102133],[8.760178,49.7102516],[8.7600914,49.7103478],[8.7599721,49.7103028]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[743238,2967416],[743265.625,2967416],[743265,2967450],[743238,2967450],[743238,2967416]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 0, "ExteriorRing": {"type":"LineString","coordinates":[[10,10],[11,10],[11,11],[10,11],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "NumInteriorRings": 1, "ExteriorRing": {"type":"LineString","coordinates":[[35,10],[45,45],[15,40],[10,20],[35,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
new file mode 100644
index 0000000..7967722
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/datatype/result.9.adm
@@ -0,0 +1,4 @@
+{ "Length": 0.004058119099397876, "Boundary": {"type":"MultiPoint","coordinates":[[-69.1991349,-12.6006222],[-69.1975081,-12.6010968]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 78.9292222699217, "Boundary": {"type":"MultiPoint","coordinates":[[10,10],[10,40],[40,40],[30,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 0.0031622776601655037, "Boundary": {"type":"MultiPoint","coordinates":[[-113.98,39.198],[-113.981,39.195]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Length": 8.48528137423857, "Boundary": {"type":"MultiPoint","coordinates":[[1,2],[7,8]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
new file mode 100644
index 0000000..a1e16dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.10.adm
@@ -0,0 +1 @@
+{"type":"Polygon","coordinates":[[[-7,4.2],[-7.1,5],[-7.1,4.3],[-7,4.2]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.13.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
new file mode 100644
index 0000000..f32a580
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.14.adm
@@ -0,0 +1 @@
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/single-method/result.7.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
new file mode 100644
index 0000000..f884259
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.15.adm
@@ -0,0 +1,10 @@
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 1.4142135623730951, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": true, "Crosses": false, "Disjoint": false, "Equals": true, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 2.8284271247461903, "Intersects": false, "Contains": false, "Crosses": false, "Disjoint": true, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": true, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": true, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": false, "Within": true }
+{ "Distance": 0.0, "Intersects": true, "Contains": false, "Crosses": false, "Disjoint": false, "Equals": false, "Overlaps": false, "Relate": false, "Touches": true, "Within": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
new file mode 100644
index 0000000..d9b7206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/geojson/two-geometries/result.16.adm
@@ -0,0 +1,10 @@
+{ "Union": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,0],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[-1,-1],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[-1,-1],[2,3.5],[1,3],[1,2],[2,1]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"LineString","coordinates":[[1,1],[2,2],[3,3],[4,4],[5,5],[6,6]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"LineString","coordinates":[[0,0],[5,5],[10,10]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null}, "Intersection": {"type":"MultiPolygon","coordinates":[],"crs":null}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,2],[3,4]],[[5,6],[7,8]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,3]],[[3,3],[4,2],[5,2]],[[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"LineString","coordinates":[[2,2],[3,3]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[0,2],[1,2],[2,2],[1,1]],[[5,2],[4,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null}, "Intersection": {"type":"Point","coordinates":[2,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"MultiLineString","coordinates":[[[1,1],[2,2]],[[0,2],[1,2],[2,2]],[[2,2],[3,2],[4,2],[5,2]],[[2,2],[3,3],[4,4],[5,5],[6,6]]],"crs":null} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Polygon","coordinates":[[[0.25,0.25],[0.5,0.25],[0.5,0.5],[0.25,0.5],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"Polygon","coordinates":[[[0,0],[1,0],[1,1],[0,1],[0,0]],[[0.25,0.25],[0.25,0.5],[0.5,0.5],[0.5,0.25],[0.25,0.25]]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
+{ "Union": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "Intersection": {"type":"Point","coordinates":[0,2],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}}, "SymDifference": {"type":"LineString","coordinates":[[0,0],[1,1],[0,2]],"crs":{"type":"name","properties":{"name":"EPSG:4326"}}} }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
new file mode 100644
index 0000000..9c4866f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5, 5
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$c])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$15(ASC), $$16(ASC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$15, $$16, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$21, $$22, 2, $$21, $$22, TRUE, TRUE, TRUE) condition (and(lt($$c.getField(2), 150), lt($$c.getField(5), 10000))) limit 10
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$21) (ASC, $$22)
+ -- STABLE_SORT [$$21(ASC), $$22(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$21, $$22])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$20, $$21, $$22] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$19, TRUE, FALSE, FALSE)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$19] <- [150]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
new file mode 100644
index 0000000..3d466ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup-select/push-limit-to-primary-lookup-select.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 32, "l_partkey": 12, "l_suppkey": 6, "l_linenumber": 6, "l_quantity": 6, "l_extendedprice": 5472.06, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-21", "l_commitdate": "1995-09-23", "l_receiptdate": "1995-07-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " gifts cajole carefully." }
+{ "l_orderkey": 33, "l_partkey": 138, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 5, "l_extendedprice": 5190.65, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1993-12-25", "l_receiptdate": "1993-12-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": ". stealthily bold exc" }
+{ "l_orderkey": 34, "l_partkey": 170, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 6, "l_extendedprice": 6421.02, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-30", "l_commitdate": "1998-09-20", "l_receiptdate": "1998-11-05", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ar foxes sleep " }
+{ "l_orderkey": 35, "l_partkey": 121, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 7, "l_extendedprice": 7147.84, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-19", "l_commitdate": "1995-12-22", "l_receiptdate": "1996-01-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " the carefully regular " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
new file mode 100644
index 0000000..29357ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.3.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5, 5
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$c])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$12(ASC), $$13(ASC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$12, $$13, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$17, $$18, 2, $$17, $$18, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ order (ASC, $$17) (ASC, $$18)
+ -- STABLE_SORT [$$17(ASC), $$18(ASC)] |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ project ([$$17, $$18])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ unnest-map [$$16, $$17, $$18] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$15, TRUE, FALSE, FALSE)
+ -- BTREE_SEARCH |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ assign [$$15] <- [150]
+ -- ASSIGN |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
new file mode 100644
index 0000000..eeedd56
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.4.adm
@@ -0,0 +1,5 @@
+{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
new file mode 100644
index 0000000..b60a0ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5, 5
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$paper])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$12(ASC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$12, $$paper] <- test.DBLP1 condition (contains($$paper.getField(1), "kimL89")) limit 10
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
new file mode 100644
index 0000000..afc422a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.4.adm
@@ -0,0 +1,5 @@
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
new file mode 100644
index 0000000..daeeeb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.3.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 5, 5
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([$$paper])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$10(ASC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$10, $$paper] <- test.DBLP1 limit 10
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
new file mode 100644
index 0000000..e9f7751
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.4.adm
@@ -0,0 +1,5 @@
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
new file mode 100644
index 0000000..91a2dfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/secondary-index-index-only/secondary-index-index-only.3.regexadm
@@ -0,0 +1 @@
+.*"processedObjects":1.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
new file mode 100644
index 0000000..8b095a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/case_08/case_08.1.adm
@@ -0,0 +1 @@
+{ "v1": true, "v2": true, "v3": [ true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
new file mode 100644
index 0000000..4be3e9a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.1.adm
@@ -0,0 +1 @@
+{ "t1": 6, "t2": 6, "t3": 6, "t4": 6, "t5": [ true, true ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
index 920f7b4..cf13212 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/is/is.1.adm
@@ -1 +1 @@
-{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false }
+{ "1": true, "3": false, "4": false, "6": true, "7": false, "8": true, "9": false, "10": true, "11": false, "12": true, "13": true, "14": true, "15": false, "16": false, "17": false, "18": true, "19": false, "20": false, "21": true, "22": true, "23": true, "24": false, "25": false, "26": false, "27": true, "28": true, "29": true, "30": false }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
new file mode 100644
index 0000000..abab248
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/const/const.1.adm
@@ -0,0 +1 @@
+{ "e": 2.71, "pi": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
new file mode 100644
index 0000000..e1310b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/degrees/degrees.1.adm
@@ -0,0 +1 @@
+{ "t1": 180 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
index 47c4e3c..775745b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.1.adm
@@ -1 +1 @@
-{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
+{ "result1": 2.0, "result2": 1.0, "result3": 0.6666666666666666, "result4": -0.5, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
new file mode 100644
index 0000000..c3d22c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int16/divide_int16.2.adm
@@ -0,0 +1 @@
+{ "result1": 2, "result2": 1, "result3": 0, "result4": 0, "result5": -0.36363637, "result6": -0.3076923076923077, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
index 83c2d70..890417b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.1.adm
@@ -1 +1 @@
-{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
+{ "result1": 3.0, "result2": 1.5, "result3": 1.0, "result4": -0.75, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
new file mode 100644
index 0000000..788a291
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int32/divide_int32.2.adm
@@ -0,0 +1 @@
+{ "result1": 3, "result2": 1, "result3": 1, "result4": 0, "result5": -0.54545456, "result6": -0.46153846153846156, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
index a6fea08..582b44d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.1.adm
@@ -1 +1 @@
-{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
+{ "result1": -4.0, "result2": -2.0, "result3": -1.3333333333333333, "result4": 1.0, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
new file mode 100644
index 0000000..af75a4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int64/divide_int64.2.adm
@@ -0,0 +1 @@
+{ "result1": -4, "result2": -2, "result3": -1, "result4": 1, "result5": 0.72727275, "result6": 0.6153846153846154, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
index 1def226..591ed6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.1.adm
@@ -1 +1 @@
-{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
+{ "result1": 1.0, "result2": 0.5, "result3": 0.3333333333333333, "result4": -0.25, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
new file mode 100644
index 0000000..f232ecf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/divide_int8/divide_int8.2.adm
@@ -0,0 +1 @@
+{ "result1": 1, "result2": 0, "result3": 0, "result4": 0, "result5": -0.18181819, "result6": -0.15384615384615385, "result7": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
new file mode 100644
index 0000000..6091a8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/numeric/radians/radians.1.adm
@@ -0,0 +1 @@
+{ "t1": 3.14 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
new file mode 100644
index 0000000..a2b8b2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/reverse/reverse.1.adm
@@ -0,0 +1 @@
+{ "t1": [ true, true ], "t2": "", "t3": "dcba", "t4": [ 98, 8457, 247, 8451, 215, 97 ], "t5": [ "ba", "cba", "dcba" ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
index ac9dedd..326e22f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr01/substr01.1.adm
@@ -1 +1 @@
-{ "str2": "ld", "str4": "g", "str6": "", "str8": "This is a test string", "str10": "This is a test string", "str13": "gThis is a another test string", "str14": "Irvine" }
+{ "str2": "ld", "str4": "g", "str6": "", "str8": "This is a test string", "str10": "string", "str13": "gThis is a another test string", "str14": "Irvine" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
index 5b0c7b9..197a7af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substring2-4/substring2-4.1.adm
@@ -1 +1 @@
-{ "result1": "HEllow" }
+{ "result1": "low" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
index b92a705..a8e64f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring/substring.1.adm
@@ -1 +1 @@
-[ "ab", "ab", "ab", "bc", "cd" ]
\ No newline at end of file
+[ "g", "ab", "ab", "bc", "cd" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
index e341861..a11b25b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/stringoffset/substring2/substring2.1.adm
@@ -1 +1 @@
-[ "abcdefg", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
\ No newline at end of file
+[ "g", "abcdefg", "abcdefg", "bcdefg", "cdefg" ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
index c76f17a..d99fd3a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.1.adm
@@ -1,2 +1 @@
-0.1
--0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
new file mode 100644
index 0000000..13a1d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/user-defined-functions/udf25/udf25.2.adm
@@ -0,0 +1 @@
+-0.25
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
new file mode 100644
index 0000000..90fd039
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int16/divide_int16.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [result1]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result2]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result3]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result4]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result5]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.float@1[
+ LiteralExpr [STRING] [-5.5f]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result6]
+ :
+ OperatorExpr [
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ div
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result7]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ LiteralExpr [NULL]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result8]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ FieldAccessor [
+ RecordConstructor [
+ ]
+ Field=a
+ ]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
new file mode 100644
index 0000000..05bd4a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int32/divide_int32.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [result1]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result2]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result3]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result4]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result5]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.float@1[
+ LiteralExpr [STRING] [-5.5f]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result6]
+ :
+ OperatorExpr [
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ div
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result7]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ LiteralExpr [NULL]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result8]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ FieldAccessor [
+ RecordConstructor [
+ ]
+ Field=a
+ ]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
new file mode 100644
index 0000000..5e1007a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int64/divide_int64.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [result1]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result2]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result3]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result4]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result5]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.float@1[
+ LiteralExpr [STRING] [-5.5f]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result6]
+ :
+ OperatorExpr [
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ div
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result7]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ LiteralExpr [NULL]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result8]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ FieldAccessor [
+ RecordConstructor [
+ ]
+ Field=a
+ ]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
new file mode 100644
index 0000000..6442d38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/divide_int8/divide_int8.2.ast
@@ -0,0 +1,107 @@
+Query:
+RecordConstructor [
+ (
+ LiteralExpr [STRING] [result1]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result2]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.int16@1[
+ LiteralExpr [STRING] [2]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result3]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.int32@1[
+ LiteralExpr [STRING] [+3]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result4]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.int64@1[
+ LiteralExpr [STRING] [-4]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result5]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.float@1[
+ LiteralExpr [STRING] [-5.5f]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result6]
+ :
+ OperatorExpr [
+ FunctionCall null.int8@1[
+ LiteralExpr [STRING] [+1]
+ ]
+ div
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result7]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ LiteralExpr [NULL]
+ ]
+ )
+ (
+ LiteralExpr [STRING] [result8]
+ :
+ OperatorExpr [
+ FunctionCall null.double@1[
+ LiteralExpr [STRING] [-6.5d]
+ ]
+ div
+ FieldAccessor [
+ RecordConstructor [
+ ]
+ Field=a
+ ]
+ ]
+ )
+]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
index a86d0a4..9d1c274 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substr01/substr01.3.ast
@@ -37,7 +37,7 @@
:
FunctionCall null.substring@2[
LiteralExpr [STRING] [This is a test string]
- - LiteralExpr [LONG] [1]
+ - LiteralExpr [LONG] [6]
]
)
(
@@ -63,7 +63,7 @@
FunctionCall null.string-length@1[
LiteralExpr [STRING] [UC Irvine]
]
- /
+ div
LiteralExpr [LONG] [2]
]
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
index 414ad6e..8ff93d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/string/substring2-4/substring2-4.3.ast
@@ -6,7 +6,7 @@
:
FunctionCall test.substring@2[
LiteralExpr [STRING] [HEllow]
- - LiteralExpr [LONG] [1]
+ - LiteralExpr [LONG] [3]
]
)
]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 5c6b100..4dd6d97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1668,7 +1668,7 @@
<test-case FilePath="dml">
<compilation-unit name="insert-with-autogenerated-pk_adm_02">
<output-dir compare="Text">insert-with-autogenerated-pk_adm_02</output-dir>
- <expected-error>Duplicate field name "id"</expected-error>
+ <expected-error>ASX1006: Duplicate field name "id"</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -1689,13 +1689,13 @@
<test-case FilePath="dml">
<compilation-unit name="load-with-autogenerated-pk_adm_02">
<output-dir compare="Text">load-with-autogenerated-pk_adm_02</output-dir>
- <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+ <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
<compilation-unit name="load-with-autogenerated-pk_adm_03">
<output-dir compare="Text">load-with-autogenerated-pk_adm_03</output-dir>
- <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id [HyracksDataException]</expected-error>
+ <expected-error>ASX3058: This record is closed, you can not add extra fields! new field name: id</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -1716,7 +1716,7 @@
<test-case FilePath="dml">
<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 [CompilationException]</expected-error>
+ <expected-error>ASX1014: Field "not_id" is not found</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="dml">
@@ -4477,7 +4477,7 @@
<test-case FilePath="numeric">
<compilation-unit name="query-issue355">
<output-dir compare="Text">query-issue355</output-dir>
- <expected-error>java.lang.NumberFormatException: For input string: "10000000000000000000"</expected-error>
+ <expected-error>For input string: "10000000000000000000"</expected-error>
</compilation-unit>
</test-case>
</test-group>
@@ -4923,7 +4923,7 @@
<output-dir compare="Text">alltypes_01</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="scan">
+ <!--test-case FilePath="scan">
<compilation-unit name="alltypes_01">
<parameter name="wrapper-array" value="true" />
<output-dir compare="Text">alltypes_01-wrapped</output-dir>
@@ -4944,7 +4944,7 @@
<compilation-unit name="alltypes_01">
<output-dir compare="Clean-JSON">alltypes_01-cleanjson</output-dir>
</compilation-unit>
- </test-case>
+ </test-case-->
<test-case FilePath="scan">
<compilation-unit name="alltypes_02">
<output-dir compare="Text">alltypes_02</output-dir>
@@ -6525,19 +6525,19 @@
<test-case FilePath="load">
<compilation-unit name="csv_05"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_05</output-dir>
- <expected-error>java.io.IOException: At record:</expected-error>
+ <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_06"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_06</output-dir>
- <expected-error>java.io.IOException: At record:</expected-error>
+ <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_07"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_07</output-dir>
- <expected-error>java.io.IOException: At record:</expected-error>
+ <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
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 2ee294a..f972050 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -19,6 +19,7 @@
<!DOCTYPE test-suite [
<!ENTITY ObjectsQueries SYSTEM "queries_sqlpp/objects/ObjectsQueries.xml">
<!ENTITY AsyncDeferredQueries SYSTEM "queries_sqlpp/async-deferred/AsyncDeferredQueries.xml">
+ <!ENTITY GeoQueries SYSTEM "queries_sqlpp/geojson/GeoJSONQueries.xml">
]>
<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
&AsyncDeferredQueries;
@@ -1263,6 +1264,32 @@
<output-dir compare="Text">least_mixed</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="missingif">
+ <output-dir compare="Text">missingif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="nullif">
+ <output-dir compare="Text">nullif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="nanif">
+ <output-dir compare="Text">nanif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="posinfif">
+ <output-dir compare="Text">posinfif</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="comparison">
+ <compilation-unit name="neginfif">
+ <output-dir compare="Text">neginfif</output-dir>
+ </compilation-unit>
+ </test-case>
+
</test-group>
<test-group name="constructor">
<test-case FilePath="constructor">
@@ -3604,6 +3631,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="misc">
+ <compilation-unit name="case_08">
+ <output-dir compare="Text">case_08</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
<compilation-unit name="dataset_nodegroup">
<output-dir compare="Text">dataset_nodegroup</output-dir>
</compilation-unit>
@@ -3657,6 +3689,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="misc">
+ <compilation-unit name="random">
+ <output-dir compare="Text">random</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
<compilation-unit name="stable_sort">
<output-dir compare="Text">stable_sort</output-dir>
</compilation-unit>
@@ -4420,6 +4457,11 @@
<output-dir compare="Text">rtree-secondary-index-open</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="nested-index/index-selection">
+ <compilation-unit name="rtree-secondary-index-optional">
+ <output-dir compare="Text">rtree-secondary-index-open</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="nested-index/external-indexing">
<test-case FilePath="nested-index/external-indexing">
@@ -4602,6 +4644,12 @@
<output-dir compare="Text">nestrecord</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="nestrecords">
+ <compilation-unit name="nested-optional-pk">
+ <output-dir compare="Text">nested-optional-pk</output-dir>
+ <expected-error>ASX1021: The primary key field "nested.id" cannot be nullable</expected-error>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="null-missing">
<test-case FilePath="null-missing">
@@ -4802,6 +4850,16 @@
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
+ <compilation-unit name="const">
+ <output-dir compare="Text">const</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
+ <compilation-unit name="degrees">
+ <output-dir compare="Text">degrees</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
<compilation-unit name="divide_double">
<output-dir compare="Text">divide_double</output-dir>
</compilation-unit>
@@ -4912,6 +4970,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="numeric">
+ <compilation-unit name="radians">
+ <output-dir compare="Text">radians</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="numeric">
<compilation-unit name="round-half-to-even0">
<output-dir compare="Text">round-half-to-even0</output-dir>
</compilation-unit>
@@ -6238,6 +6301,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="string">
+ <compilation-unit name="reverse">
+ <output-dir compare="Text">reverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
<compilation-unit name="rtrim">
<output-dir compare="Text">rtrim</output-dir>
</compilation-unit>
@@ -9330,6 +9398,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="filters">
+ <compilation-unit name="filter-auto-key">
+ <output-dir compare="Text">filter-auto-key</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="filters">
<compilation-unit name="load">
<output-dir compare="Text">load</output-dir>
</compilation-unit>
@@ -9796,4 +9869,27 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="limit">
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-scan">
+ <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-scan-select">
+ <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup">
+ <output-dir compare="Text">push-limit-to-primary-lookup</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="limit">
+ <compilation-unit name="push-limit-to-primary-lookup-select">
+ <output-dir compare="Text">push-limit-to-primary-lookup-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ &GeoQueries;
</test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
new file mode 100644
index 0000000..7075739
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetInfoProvider.java
@@ -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.
+ */
+package org.apache.asterix.common.api;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.hyracks.api.application.INCServiceContext;
+
+@FunctionalInterface
+public interface IDatasetInfoProvider extends Serializable {
+ /**
+ * Get the dataset info
+ *
+ * @param serviceCtx
+ * the nc service context
+ * @return the dataset info object
+ */
+ DatasetInfo getDatasetInfo(INCServiceContext serviceCtx);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
index 9551935..7e9879a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IExtension.java
@@ -40,7 +40,11 @@
/**
* Extends Language Syntax and Algebric Operations
*/
- LANG
+ LANG,
+ /**
+ * Extends Recovery Capabilities
+ */
+ RECOVERY
}
/**
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.java
new file mode 100644
index 0000000..5fd37e9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ILSMComponentIdGeneratorFactory.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.common.api;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.common.IResource;
+
+@FunctionalInterface
+public interface ILSMComponentIdGeneratorFactory extends Serializable {
+ /**
+ * Get the Id generator for the resource
+ *
+ * @param serviceCtx
+ * @param resource
+ * @return
+ * @throws HyracksDataException
+ */
+ ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
+ throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
index 9eb6259..b6581ec2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/INcApplicationContext.java
@@ -27,6 +27,7 @@
import org.apache.asterix.common.replication.IReplicationManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.IReplicaManager;
+import org.apache.asterix.common.transactions.IRecoveryManagerFactory;
import org.apache.asterix.common.transactions.ITransactionSubsystem;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.INCServiceContext;
@@ -65,7 +66,8 @@
IResourceIdFactory getResourceIdFactory();
- void initialize(boolean initialRun) throws IOException, AlgebricksException;
+ void initialize(IRecoveryManagerFactory recoveryManagerFactory, boolean initialRun)
+ throws IOException, AlgebricksException;
void setShuttingdown(boolean b);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
index c15f8a7..4ee00cd 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
@@ -38,7 +38,7 @@
TXN_LOG_BUFFER_NUMPAGES(INTEGER, 8, "The number of pages in the transaction log tail"),
TXN_LOG_BUFFER_PAGESIZE(
INTEGER_BYTE_UNIT,
- StorageUtil.getIntSizeInBytes(128, KILOBYTE),
+ StorageUtil.getIntSizeInBytes(4, MEGABYTE),
"The page size (in bytes) for transaction log buffer"),
TXN_LOG_PARTITIONSIZE(
LONG_BYTE_UNIT,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
index b7af0b6..3ff13cb 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseOperationTracker.java
@@ -37,8 +37,7 @@
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
- || opType == LSMOperationType.REPLICATE) {
+ if (opType == LSMOperationType.REPLICATE) {
dsInfo.declareActiveIOOperation();
}
}
@@ -47,14 +46,14 @@
public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
if (opType == LSMOperationType.REPLICATE) {
- dsInfo.undeclareActiveIOOperation();
+ completeOperation(index, opType, searchCallback, modificationCallback);
}
}
@Override
public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
- if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+ if (opType == LSMOperationType.REPLICATE) {
dsInfo.undeclareActiveIOOperation();
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
index 41461ec..a14fac8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/CorrelatedPrefixMergePolicy.java
@@ -125,7 +125,7 @@
}
}
ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleMerge(lsmIndex.getIOOperationCallback(), mergableComponents);
+ accessor.scheduleMerge(mergableComponents);
}
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
index f4d764a..6e2e320 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfo.java
@@ -55,7 +55,7 @@
this.setRegistered(false);
this.setMemoryAllocated(false);
this.logManager = logManager;
- waitLog.setLogType(LogType.WAIT);
+ waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
waitLog.computeAndSetLogSize();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
new file mode 100644
index 0000000..597b5da
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetInfoProvider.java
@@ -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.
+ */
+package org.apache.asterix.common.context;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public class DatasetInfoProvider implements IDatasetInfoProvider {
+
+ private static final long serialVersionUID = 1L;
+ private final int datasetId;
+
+ public DatasetInfoProvider(int datasetId) {
+ this.datasetId = datasetId;
+ }
+
+ @Override
+ public DatasetInfo getDatasetInfo(INCServiceContext serviceCtx) {
+ IDatasetLifecycleManager dslcManager =
+ ((INcApplicationContext) serviceCtx.getApplicationContext()).getDatasetLifecycleManager();
+ return dslcManager.getDatasetInfo(datasetId);
+ }
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
index 83e3144..85eacf8 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLSMComponentIdGeneratorFactory.java
@@ -20,12 +20,12 @@
package org.apache.asterix.common.context;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.hyracks.storage.common.IResource;
/**
@@ -51,5 +51,4 @@
int partition = StoragePathUtil.getPartitionNumFromRelativePath(resource.getPath());
return dslcManager.getComponentIdGenerator(datasetId, partition);
}
-
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index b715eec..50a4bef 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -23,7 +23,6 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -33,26 +32,24 @@
import org.apache.asterix.common.api.IDatasetMemoryManager;
import org.apache.asterix.common.config.StorageProperties;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.dataflow.LSMIndexUtil;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
import org.apache.asterix.common.replication.IReplicationStrategy;
import org.apache.asterix.common.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.transactions.ILogManager;
import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogType;
import org.apache.asterix.common.utils.StoragePathUtil;
-import org.apache.asterix.common.utils.TransactionUtil;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
import org.apache.hyracks.storage.common.IIndex;
import org.apache.hyracks.storage.common.ILocalResourceRepository;
@@ -68,7 +65,7 @@
private final ILocalResourceRepository resourceRepository;
private final IDatasetMemoryManager memoryManager;
private final ILogManager logManager;
- private final LogRecord logRecord;
+ private final LogRecord waitLog;
private final int numPartitions;
private volatile boolean stopped = false;
private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
@@ -82,7 +79,9 @@
this.memoryManager = memoryManager;
this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
this.numPartitions = numPartitions;
- logRecord = new LogRecord();
+ waitLog = new LogRecord();
+ waitLog.setLogType(LogType.WAIT_FOR_FLUSHES);
+ waitLog.computeAndSetLogSize();
}
@Override
@@ -348,7 +347,7 @@
}
private void populateOpTrackerAndIdGenerator(DatasetResource dataset, int partition) {
- ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
+ ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(storageProperties.getMemoryComponentsNum());
PrimaryIndexOperationTracker opTracker = new PrimaryIndexOperationTracker(dataset.getDatasetID(), partition,
logManager, dataset.getDatasetInfo(), idGenerator);
dataset.setPrimaryIndexOperationTracker(partition, opTracker);
@@ -369,7 +368,9 @@
@Override
public synchronized void flushAllDatasets() throws HyracksDataException {
for (DatasetResource dsr : datasets.values()) {
- flushDatasetOpenIndexes(dsr, false);
+ if (dsr.getDatasetInfo().isOpen()) {
+ flushDatasetOpenIndexes(dsr, false);
+ }
}
}
@@ -398,11 +399,10 @@
PrimaryIndexOperationTracker opTracker, long targetLSN) throws HyracksDataException {
int partition = opTracker.getPartition();
for (ILSMIndex lsmIndex : dsr.getDatasetInfo().getDatasetPartitionOpenIndexes(partition)) {
- AbstractLSMIOOperationCallback ioCallback =
- (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+ LSMIOOperationCallback ioCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
if (!(lsmIndex.isCurrentMutableComponentEmpty() || ioCallback.hasPendingFlush()
|| opTracker.isFlushLogCreated() || opTracker.isFlushOnExit())) {
- long firstLSN = ioCallback.getFirstLSN();
+ long firstLSN = ioCallback.getPersistenceLsn();
if (firstLSN < targetLSN) {
LOGGER.info("Checkpoint flush dataset {} partition {}", dsr.getDatasetID(), partition);
opTracker.setFlushOnExit(true);
@@ -422,90 +422,48 @@
*/
private void flushDatasetOpenIndexes(DatasetResource dsr, boolean asyncFlush) throws HyracksDataException {
DatasetInfo dsInfo = dsr.getDatasetInfo();
+ if (!dsInfo.isOpen()) {
+ throw new IllegalStateException("flushDatasetOpenIndexes is called on a dataset that is closed");
+ }
if (dsInfo.isExternal()) {
// no memory components for external dataset
return;
}
+ // ensure all in-flight flushes gets scheduled
+ logManager.log(waitLog);
for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
// flush each partition one by one
if (primaryOpTracker.getNumActiveOperations() > 0) {
throw new IllegalStateException(
"flushDatasetOpenIndexes is called on a dataset with currently active operations");
}
- int partition = primaryOpTracker.getPartition();
- Collection<ILSMIndex> indexes = dsInfo.getDatasetPartitionOpenIndexes(partition);
- ILSMIndex flushIndex = null;
- for (ILSMIndex lsmIndex : indexes) {
- if (!lsmIndex.isCurrentMutableComponentEmpty()) {
- flushIndex = lsmIndex;
- break;
- }
- }
- if (flushIndex == null) {
- // all open indexes are empty, nothing to flush
- continue;
- }
- LSMComponentId componentId = (LSMComponentId) flushIndex.getCurrentMemoryComponent().getId();
- ILSMComponentIdGenerator idGenerator = getComponentIdGenerator(dsInfo.getDatasetID(), partition);
- idGenerator.refresh();
-
- if (dsInfo.isDurable()) {
-
- synchronized (logRecord) {
- TransactionUtil.formFlushLogRecord(logRecord, dsInfo.getDatasetID(), partition,
- componentId.getMinId(), componentId.getMaxId(), null);
- try {
- logManager.log(logRecord);
- } catch (ACIDException e) {
- throw new HyracksDataException("could not write flush log while closing dataset", e);
- }
-
- try {
- //notification will come from LogBuffer class (notifyFlushTerminator)
- logRecord.wait();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw HyracksDataException.create(e);
- }
- }
- }
- for (ILSMIndex index : indexes) {
- //update resource lsn
- AbstractLSMIOOperationCallback ioOpCallback =
- (AbstractLSMIOOperationCallback) index.getIOOperationCallback();
- ioOpCallback.updateLastLSN(logRecord.getLSN());
- }
-
- if (asyncFlush) {
- for (ILSMIndex index : indexes) {
- ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFlush(index.getIOOperationCallback());
- }
- } else {
- for (ILSMIndex index : indexes) {
- // TODO: This is not efficient since we flush the indexes sequentially.
- // Think of a way to allow submitting the flush requests concurrently.
- // We don't do them concurrently because this may lead to a deadlock scenario
- // between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
- ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFlush(index.getIOOperationCallback());
- // Wait for the above flush op.
- dsInfo.waitForIO();
- }
- }
+ primaryOpTracker.setFlushOnExit(true);
+ primaryOpTracker.flushIfNeeded();
}
-
+ // ensure requested flushes were scheduled
+ logManager.log(waitLog);
+ if (!asyncFlush) {
+ List<FlushOperation> flushes = new ArrayList<>();
+ for (PrimaryIndexOperationTracker primaryOpTracker : dsr.getOpTrackers()) {
+ flushes.addAll(primaryOpTracker.getScheduledFlushes());
+ }
+ LSMIndexUtil.waitFor(flushes);
+ }
}
private void closeDataset(DatasetResource dsr) throws HyracksDataException {
// First wait for any ongoing IO operations
DatasetInfo dsInfo = dsr.getDatasetInfo();
- dsInfo.waitForIO();
try {
flushDatasetOpenIndexes(dsr, false);
} catch (Exception e) {
throw HyracksDataException.create(e);
}
+ // wait for merges that were scheduled due to the above flush
+ // ideally, we shouldn't need this since merges should still work.
+ // They don't need a special memory budget but there is a problem
+ // for some merge policies that need to access dataset info (correlated prefix)
+ dsInfo.waitForIO();
for (IndexInfo iInfo : dsInfo.getIndexes().values()) {
closeIndex(iInfo);
}
@@ -517,7 +475,9 @@
public synchronized void closeAllDatasets() throws HyracksDataException {
ArrayList<DatasetResource> openDatasets = new ArrayList<>(datasets.values());
for (DatasetResource dsr : openDatasets) {
- closeDataset(dsr);
+ if (dsr.isOpen()) {
+ closeDataset(dsr);
+ }
}
}
@@ -624,7 +584,7 @@
@Override
public void flushDataset(IReplicationStrategy replicationStrategy) throws HyracksDataException {
for (DatasetResource dsr : datasets.values()) {
- if (replicationStrategy.isMatch(dsr.getDatasetID())) {
+ if (dsr.isOpen() && replicationStrategy.isMatch(dsr.getDatasetID())) {
flushDatasetOpenIndexes(dsr, false);
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 74848d5..52c8962 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -19,11 +19,16 @@
package org.apache.asterix.common.context;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
import org.apache.asterix.common.transactions.AbstractOperationCallback;
import org.apache.asterix.common.transactions.ILogManager;
@@ -33,18 +38,21 @@
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
-public class PrimaryIndexOperationTracker extends BaseOperationTracker {
-
+public class PrimaryIndexOperationTracker extends BaseOperationTracker implements IoOperationCompleteListener {
private final int partition;
// Number of active operations on an ILSMIndex instance.
private final AtomicInteger numActiveOperations;
@@ -52,6 +60,7 @@
private final ILSMComponentIdGenerator idGenerator;
private boolean flushOnExit = false;
private boolean flushLogCreated = false;
+ private final Map<String, FlushOperation> scheduledFlushes = new HashMap<>();
public PrimaryIndexOperationTracker(int datasetID, int partition, ILogManager logManager, DatasetInfo dsInfo,
ILSMComponentIdGenerator idGenerator) {
@@ -65,20 +74,9 @@
@Override
public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
IModificationOperationCallback modificationCallback) throws HyracksDataException {
+ super.beforeOperation(index, opType, searchCallback, modificationCallback);
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
incrementNumActiveOperations(modificationCallback);
- } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
- || opType == LSMOperationType.REPLICATE) {
- dsInfo.declareActiveIOOperation();
- }
- }
-
- @Override
- public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
- IModificationOperationCallback modificationCallback) throws HyracksDataException {
- // Searches are immediately considered complete, because they should not prevent the execution of flushes.
- if (opType == LSMOperationType.REPLICATE) {
- completeOperation(index, opType, searchCallback, modificationCallback);
}
}
@@ -86,12 +84,10 @@
public synchronized void completeOperation(ILSMIndex index, LSMOperationType opType,
ISearchOperationCallback searchCallback, IModificationOperationCallback modificationCallback)
throws HyracksDataException {
+ super.completeOperation(index, opType, searchCallback, modificationCallback);
if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
decrementNumActiveOperations(modificationCallback);
flushIfNeeded();
- } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE
- || opType == LSMOperationType.REPLICATE) {
- dsInfo.undeclareActiveIOOperation();
}
}
@@ -118,15 +114,33 @@
}
if (needsFlush || flushOnExit) {
+ flushOnExit = false;
// make the current mutable components READABLE_UNWRITABLE to stop coming modify operations from entering
// them until the current flush is scheduled.
LSMComponentId primaryId = null;
+ //Double check that the primary index has been modified
+ synchronized (this) {
+ if (numActiveOperations.get() > 0) {
+ throw new IllegalStateException(
+ "Can't request a flush on an index with active operations: " + numActiveOperations.get());
+ }
+ for (ILSMIndex lsmIndex : indexes) {
+ if (lsmIndex.isPrimaryIndex() && lsmIndex.isCurrentMutableComponentEmpty()) {
+ return;
+ }
+ }
+ }
for (ILSMIndex lsmIndex : indexes) {
ILSMOperationTracker opTracker = lsmIndex.getOperationTracker();
synchronized (opTracker) {
ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
+ if (memComponent.getWriterCount() > 0) {
+ throw new IllegalStateException(
+ "Can't request a flush on a component with writers inside: Index:" + lsmIndex
+ + " Component:" + memComponent);
+ }
if (memComponent.getState() == ComponentState.READABLE_WRITABLE && memComponent.isModified()) {
- memComponent.setState(ComponentState.READABLE_UNWRITABLE);
+ memComponent.setUnwritable();
}
if (lsmIndex.isPrimaryIndex()) {
primaryId = (LSMComponentId) memComponent.getId();
@@ -137,7 +151,6 @@
throw new IllegalStateException("Primary index not found in dataset " + dsInfo.getDatasetID());
}
LogRecord logRecord = new LogRecord();
- flushOnExit = false;
if (dsInfo.isDurable()) {
/*
* Generate a FLUSH log.
@@ -171,21 +184,41 @@
throw new IllegalStateException("Operation started while index was pending scheduling a flush");
}
idGenerator.refresh();
- for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
- //get resource
- ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- //update resource lsn
- AbstractLSMIOOperationCallback ioOpCallback =
- (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
- ioOpCallback.updateLastLSN(logRecord.getLSN());
- //schedule flush after update
- accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+ long flushLsn = logRecord.getLSN();
+ ILSMComponentId nextComponentId = idGenerator.getId();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+ synchronized (scheduledFlushes) {
+ for (ILSMIndex lsmIndex : dsInfo.getDatasetPartitionOpenIndexes(partition)) {
+ ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ accessor.getOpContext().setParameters(flushMap);
+ ILSMIOOperation flush = accessor.scheduleFlush();
+ scheduledFlushes.put(flush.getTarget().getRelativePath(), (FlushOperation) flush);
+ flush.addCompleteListener(this);
+ }
}
} finally {
flushLogCreated = false;
}
}
+ @Override
+ public void completed(ILSMIOOperation operation) {
+ synchronized (scheduledFlushes) {
+ scheduledFlushes.remove(operation.getTarget().getRelativePath());
+ }
+ }
+
+ public List<FlushOperation> getScheduledFlushes() {
+ synchronized (scheduledFlushes) {
+ Collection<FlushOperation> scheduled = scheduledFlushes.values();
+ List<FlushOperation> flushes = new ArrayList<FlushOperation>(scheduled.size());
+ flushes.addAll(scheduled);
+ return flushes;
+ }
+ }
+
public int getNumActiveOperations() {
return numActiveOperations.get();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
index e844192..71d16f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMIndexUtil.java
@@ -18,15 +18,19 @@
*/
package org.apache.asterix.common.dataflow;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import java.util.List;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.transactions.ILogManager;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
public class LSMIndexUtil {
+ private LSMIndexUtil() {
+ }
+
public static void checkAndSetFirstLSN(AbstractLSMIndex lsmIndex, ILogManager logManager)
throws HyracksDataException {
// If the index has an empty memory component, we need to set its first LSN (For soft checkpoint)
@@ -34,11 +38,21 @@
//prevent transactions from incorrectly setting the first LSN on a modified component by checking the index is still empty
synchronized (lsmIndex.getOperationTracker()) {
if (lsmIndex.isCurrentMutableComponentEmpty()) {
- AbstractLSMIOOperationCallback ioOpCallback =
- (AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback();
- ioOpCallback.setFirstLSN(logManager.getAppendLSN());
+ LSMIOOperationCallback ioOpCallback = (LSMIOOperationCallback) lsmIndex.getIOOperationCallback();
+ ioOpCallback.setFirstLsnForCurrentMemoryComponent(logManager.getAppendLSN());
}
}
}
}
+
+ public static void waitFor(List<? extends ILSMIOOperation> ioOperations) throws HyracksDataException {
+ for (int i = 0; i < ioOperations.size(); i++) {
+ try {
+ ioOperations.get(i).sync();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
index 256ce08..359054e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
@@ -20,6 +20,8 @@
import java.util.function.Predicate;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
public class ExceptionUtils {
public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
public static final String PARAMETER_NAME = "Parameter name: ";
@@ -45,13 +47,28 @@
Throwable current = e;
Throwable cause = e.getCause();
while (cause != null && cause != current) {
- Throwable nextCause = current.getCause();
current = cause;
- cause = nextCause;
+ cause = current.getCause();
}
return current;
}
+ public static Throwable getCause(Throwable e, String component, int code) {
+ Throwable current = e;
+ Throwable expected =
+ (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+ && ((HyracksDataException) current).getComponent().equals(component)) ? current : null;
+ Throwable cause = e.getCause();
+ while (cause != null && cause != current) {
+ current = cause;
+ expected =
+ (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
+ && ((HyracksDataException) current).getComponent().equals(component)) ? current : expected;
+ cause = current.getCause();
+ }
+ return expected == null ? current : expected;
+ }
+
/**
* Determines whether supplied exception contains a matching cause in its hierarchy, or is itself a match
*/
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
deleted file mode 100644
index f027979..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ /dev/null
@@ -1,287 +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.common.ioopcallbacks;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.asterix.common.storage.ResourceReference;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.LongPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
-
-// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
-public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
- public static final MutableArrayValueReference LSN_KEY = new MutableArrayValueReference("LSN".getBytes());
- public static final long INVALID = -1L;
-
- protected final ILSMIndex lsmIndex;
- // First LSN per mutable component. TODO: move from hyracks to asterixdb
- protected final long[] firstLSNs;
- // A boolean array to keep track of flush operations
- protected final boolean[] flushRequested;
- // TODO: move from hyracks to asterixdb
- protected final long[] mutableLastLSNs;
- // Index of the currently flushing or next to be flushed component
- protected int readIndex;
- // Index of the currently being written to component
- protected int writeIndex;
- // Index of the memory component to be recycled
- protected int recycleIndex;
- // Indicates whether this index has been scheduled to flush (no matter whether succeeds or not)
- protected boolean hasFlushed;
- // Keep track of the component Id of the next component being activated.
- protected ILSMComponentId[] nextComponentIds;
-
- protected final ILSMComponentIdGenerator idGenerator;
- protected final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
- private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
- private final Map<ILSMComponentId, Long> componentLsnMap = new HashMap<>();
-
- public AbstractLSMIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
- this.lsmIndex = lsmIndex;
- this.idGenerator = idGenerator;
- this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
- int count = lsmIndex.getNumberOfAllMemoryComponents();
- mutableLastLSNs = new long[count];
- firstLSNs = new long[count];
- flushRequested = new boolean[count];
- readIndex = 0;
- writeIndex = 0;
- recycleIndex = 0;
- hasFlushed = false;
- nextComponentIds = new ILSMComponentId[count];
- if (count > 0) {
- nextComponentIds[0] = idGenerator.getId();
- }
- }
-
- @Override
- public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
- /*
- * This method was called on the scheduleFlush operation.
- * We set the lastLSN to the last LSN for the index (the LSN for the flush log)
- * We mark the component flushing flag
- * We then move the write pointer to the next component and sets its first LSN to the flush log LSN
- */
- synchronized (this) {
- flushRequested[writeIndex] = true;
- writeIndex = (writeIndex + 1) % mutableLastLSNs.length;
- // Set the firstLSN of the next component unless it is being flushed
- if (writeIndex != readIndex) {
- firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
- }
-
- }
- }
- }
-
- @Override
- public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- //TODO: Copying Filters and all content of the metadata pages for flush operation should be done here
- if (opCtx.getNewComponent() == null) {
- // failed operation. Nothing to do.
- return;
- }
- putLSNIntoMetadata(opCtx.getNewComponent(), opCtx.getComponentsToBeMerged());
- putComponentIdIntoMetadata(opCtx.getIoOperationType(), opCtx.getNewComponent(),
- opCtx.getComponentsToBeMerged());
- componentLsnMap.put(opCtx.getNewComponent().getId(), getComponentLSN(opCtx.getComponentsToBeMerged()));
- if (opCtx.getIoOperationType() == LSMIOOperationType.MERGE) {
- if (opCtx.getComponentsToBeMerged().isEmpty()) {
- throw new IllegalStateException("Merge must have old components");
- }
- LongPointable markerLsn = LongPointable.FACTORY
- .createPointable(ComponentUtils.getLong(opCtx.getComponentsToBeMerged().get(0).getMetadata(),
- ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
- opCtx.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
- } else if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
- // advance memory component indexes
- synchronized (this) {
- // we've already consumed the specified LSN/component id.
- // Now we can advance to the next component
- flushRequested[readIndex] = false;
- // if the component which just finished flushing is the component that will be modified next,
- // we set its first LSN to its previous LSN
- if (readIndex == writeIndex) {
- firstLSNs[writeIndex] = mutableLastLSNs[writeIndex];
- }
- readIndex = (readIndex + 1) % mutableLastLSNs.length;
- }
- }
- }
-
- @Override
- public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // The operation was complete and the next I/O operation for the LSM index didn't start yet
- if (opCtx.getIoOperationType() == LSMIOOperationType.FLUSH) {
- hasFlushed = true;
- if (opCtx.getNewComponent() != null) {
- final Long lsn = componentLsnMap.remove(opCtx.getNewComponent().getId());
- if (lsn == null) {
- throw new IllegalStateException("Unidentified flushed component: " + opCtx.getNewComponent());
- }
- // empty component doesn't have any files
- final Optional<String> componentFile =
- opCtx.getNewComponent().getLSMComponentPhysicalFiles().stream().findAny();
- if (componentFile.isPresent()) {
- final ResourceReference ref = ResourceReference.of(componentFile.get());
- final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
- indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn);
- }
- }
- }
- }
-
- private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<? extends ILSMComponent> oldComponents)
- throws HyracksDataException {
- newComponent.getMetadata().put(LSN_KEY, LongPointable.FACTORY.createPointable(getComponentLSN(oldComponents)));
- }
-
- public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
- LongPointable pointable = new LongPointable();
- IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
- metadataPageManager.get(metadataPageManager.createMetadataFrame(), LSN_KEY, pointable);
- return pointable.getLength() == 0 ? INVALID : pointable.longValue();
- }
-
- private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
- throws HyracksDataException {
- if (mergedComponents.isEmpty()) {
- return null;
- }
- return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
- mergedComponents.get(mergedComponents.size() - 1).getId());
-
- }
-
- private void putComponentIdIntoMetadata(LSMIOOperationType opType, ILSMDiskComponent newComponent,
- List<? extends ILSMComponent> oldComponents) throws HyracksDataException {
- // the id of flushed component is set when we copy the metadata of the memory component
- if (opType == LSMIOOperationType.MERGE) {
- ILSMComponentId componentId = getMergedComponentId(oldComponents);
- LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
- }
- }
-
- public synchronized void updateLastLSN(long lastLSN) {
- if (!flushRequested[writeIndex]) {
- //if the memory component pointed by writeIndex is being flushed, we should ignore this update call
- //since otherwise the original LSN is overwritten.
- //Moreover, since the memory component is already being flushed, the next scheduleFlush request must fail.
- //See https://issues.apache.org/jira/browse/ASTERIXDB-1917
- mutableLastLSNs[writeIndex] = lastLSN;
- if (hasFlushed || lsmIndex.isMemoryComponentsAllocated()) {
- // we only (re)set next component id if either this index has been flushed (no matter succeed or not)
- // or the memory component has been allocated
- // This prevents the case where indexes in a partition are being allocated, while another partition
- // tries to schedule flush
- nextComponentIds[writeIndex] = idGenerator.getId();
- }
- }
- }
-
- /**
- * Used during the recovery process to force refresh the next component id
- */
- public void forceRefreshNextId() {
- nextComponentIds[writeIndex] = idGenerator.getId();
- }
-
- public synchronized void setFirstLSN(long firstLSN) {
- // We make sure that this method is only called on an empty component so the first LSN is not set incorrectly
- firstLSNs[writeIndex] = firstLSN;
- }
-
- public synchronized long getFirstLSN() {
- // We make sure that this method is only called on a non-empty component so the returned LSN is meaningful
- // The firstLSN is always the lsn of the currently being flushed component or the next
- // to be flushed when no flush operation is on going
- return firstLSNs[readIndex];
- }
-
- public synchronized boolean hasPendingFlush() {
- for (int i = 0; i < flushRequested.length; i++) {
- if (flushRequested[i]) {
- return true;
- }
- }
- return false;
- }
-
- public long getComponentLSN(List<? extends ILSMComponent> diskComponents) throws HyracksDataException {
- if (diskComponents.isEmpty()) {
- // Implies a flush IO operation. --> moves the flush pointer
- // Flush operation of an LSM index are executed sequentially.
- synchronized (this) {
- return mutableLastLSNs[readIndex];
- }
- }
- // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
- long maxLSN = -1L;
- for (ILSMComponent c : diskComponents) {
- DiskComponentMetadata md = ((ILSMDiskComponent) c).getMetadata();
- maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
- }
- return maxLSN;
- }
-
- private synchronized ILSMComponentId getLSMComponentId() {
- return nextComponentIds[recycleIndex];
- }
-
- @Override
- public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
- ILSMComponentId componentId = getLSMComponentId();
- component.resetId(componentId, false);
- if (componentSwitched) {
- recycleIndex = (recycleIndex + 1) % nextComponentIds.length;
- }
- }
-
- @Override
- public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
- if (component == lsmIndex.getCurrentMemoryComponent()) {
- // only set the component id for the first (current) memory component
- ILSMComponentId componentId = getLSMComponentId();
- component.resetId(componentId, false);
- }
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 5b9883c..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/AbstractLSMIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,86 +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.common.ioopcallbacks;
-
-import java.io.ObjectStreamException;
-
-import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.common.IResource;
-
-public abstract class AbstractLSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- protected ILSMComponentIdGeneratorFactory idGeneratorFactory;
-
- protected transient INCServiceContext ncCtx;
-
- protected transient IResource resource;
-
- public AbstractLSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- this.idGeneratorFactory = idGeneratorFactory;
- }
-
- @Override
- public void initialize(INCServiceContext ncCtx, IResource resource) {
- this.ncCtx = ncCtx;
- this.resource = resource;
- }
-
- protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
- return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
- }
-
- protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
- return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
- }
-
- private void readObjectNoData() throws ObjectStreamException {
- idGeneratorFactory = new ILSMComponentIdGeneratorFactory() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
- // used for backward compatibility
- // if idGeneratorFactory is not set for legacy lsm indexes, we return a default
- // component id generator which always generates the missing component id.
- return new ILSMComponentIdGenerator() {
- @Override
- public void refresh() {
- // No op
- }
-
- @Override
- public ILSMComponentId getId() {
- return LSMComponentId.MISSING_COMPONENT_ID;
- }
- };
- }
- };
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
deleted file mode 100644
index db6c609..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +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.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
- public LSMBTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider checkpointManagerProvider) {
- super(index, idGenerator, checkpointManagerProvider);
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 97badb2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +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.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- public LSMBTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- super(idGeneratorFactory);
- }
-
- @Override
- public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
- return new LSMBTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
deleted file mode 100644
index da1446b..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallback.java
+++ /dev/null
@@ -1,31 +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.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallback extends AbstractLSMIOOperationCallback {
-
- public LSMBTreeWithBuddyIOOperationCallback(ILSMIndex lsmIndex, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider checkpointManagerProvider) {
- super(lsmIndex, idGenerator, checkpointManagerProvider);
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
deleted file mode 100644
index 9b32345..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +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.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMBTreeWithBuddyIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- public LSMBTreeWithBuddyIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- super(idGeneratorFactory);
- }
-
- @Override
- public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
- return new LSMBTreeWithBuddyIOOperationCallback(index, getComponentIdGenerator(),
- getIndexCheckpointManagerProvider());
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
new file mode 100644
index 0000000..ea53d68
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIOOperationCallback.java
@@ -0,0 +1,280 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.ioopcallbacks;
+
+import java.util.ArrayDeque;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.storage.ResourceReference;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
+import org.apache.hyracks.storage.am.common.freepage.MutableArrayValueReference;
+import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
+import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+// A single LSMIOOperationCallback per LSM index used to perform actions around Flush and Merge operations
+public class LSMIOOperationCallback implements ILSMIOOperationCallback {
+ private static final Logger LOGGER = LogManager.getLogger();
+ public static final String KEY_FLUSH_LOG_LSN = "FlushLogLsn";
+ public static final String KEY_NEXT_COMPONENT_ID = "NextComponentId";
+ public static final String KEY_FLUSHED_COMPONENT_ID = "FlushedComponentId";
+ private static final String KEY_FIRST_LSN = "FirstLsn";
+ private static final MutableArrayValueReference KEY_METADATA_FLUSH_LOG_LSN =
+ new MutableArrayValueReference(KEY_FLUSH_LOG_LSN.getBytes());
+ public static final long INVALID_LSN = -1L;
+ private final ArrayBackedValueStorage buffer = new ArrayBackedValueStorage(Long.BYTES);
+ private final IIndexCheckpointManagerProvider indexCheckpointManagerProvider;
+ protected final DatasetInfo dsInfo;
+ protected final ILSMIndex lsmIndex;
+ private long firstLsnForCurrentMemoryComponent = 0L;
+ private long persistenceLsn = 0L;
+ private int pendingFlushes = 0;
+ private Deque<ILSMComponentId> componentIds = new ArrayDeque<>();
+
+ public LSMIOOperationCallback(DatasetInfo dsInfo, ILSMIndex lsmIndex, ILSMComponentId nextComponentId,
+ IIndexCheckpointManagerProvider indexCheckpointManagerProvider) {
+ this.dsInfo = dsInfo;
+ this.lsmIndex = lsmIndex;
+ this.indexCheckpointManagerProvider = indexCheckpointManagerProvider;
+ componentIds.add(nextComponentId);
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+ // No Op
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ return;
+ }
+ if (operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+ Map<String, Object> map = operation.getParameters();
+ putComponentIdIntoMetadata(operation.getNewComponent(), (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID));
+ } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ Map<String, Object> map = operation.getParameters();
+ putLSNIntoMetadata(operation.getNewComponent(), (Long) map.get(KEY_FLUSH_LOG_LSN));
+ putComponentIdIntoMetadata(operation.getNewComponent(),
+ ((FlushOperation) operation).getFlushingComponent().getId());
+ } else if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+ List<ILSMDiskComponent> mergedComponents = operation.getAccessor().getOpContext().getComponentsToBeMerged();
+ putLSNIntoMetadata(operation.getNewComponent(), mergedComponents);
+ putComponentIdIntoMetadata(operation.getNewComponent(), mergedComponents);
+ LongPointable markerLsn =
+ LongPointable.FACTORY.createPointable(ComponentUtils.getLong(mergedComponents.get(0).getMetadata(),
+ ComponentUtils.MARKER_LSN_KEY, ComponentUtils.NOT_FOUND, buffer));
+ operation.getNewComponent().getMetadata().put(ComponentUtils.MARKER_LSN_KEY, markerLsn);
+ }
+ }
+
+ @Override
+ public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ return;
+ }
+ if (operation.getIOOpertionType() != LSMIOOperationType.LOAD
+ && operation.getAccessor().getOpContext().getOperation() == IndexOperation.DELETE_COMPONENTS) {
+ deleteComponentsFromCheckpoint(operation);
+ } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+ || operation.getIOOpertionType() == LSMIOOperationType.LOAD) {
+ addComponentToCheckpoint(operation);
+ }
+ }
+
+ private void addComponentToCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+ // will always update the checkpoint file even if no new component was created
+ FileReference target = operation.getTarget();
+ Map<String, Object> map = operation.getParameters();
+ final Long lsn =
+ operation.getIOOpertionType() == LSMIOOperationType.FLUSH ? (Long) map.get(KEY_FLUSH_LOG_LSN) : 0L;
+ final LSMComponentId id = (LSMComponentId) map.get(KEY_FLUSHED_COMPONENT_ID);
+ final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+ final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(ref.getName());
+ indexCheckpointManagerProvider.get(ref).flushed(componentEndTime, lsn, id.getMaxId());
+ }
+
+ private void deleteComponentsFromCheckpoint(ILSMIOOperation operation) throws HyracksDataException {
+ // component was deleted... if a flush, do nothing.. if a merge, must update the checkpoint file
+ if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
+ // Get component id of the last disk component
+ LSMComponentId mostRecentComponentId =
+ getMostRecentComponentId(operation.getAccessor().getOpContext().getComponentsToBeMerged());
+ // Update the checkpoint file
+ FileReference target = operation.getTarget();
+ final ResourceReference ref = ResourceReference.of(target.getAbsolutePath());
+ indexCheckpointManagerProvider.get(ref).setLastComponentId(mostRecentComponentId.getMaxId());
+ } else if (operation.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+ throw new IllegalStateException("Unexpected IO operation: " + operation.getIOOpertionType());
+ }
+ }
+
+ private LSMComponentId getMostRecentComponentId(Collection<ILSMDiskComponent> deletedComponents)
+ throws HyracksDataException {
+ // must sync on opTracker to ensure list of components doesn't change
+ synchronized (lsmIndex.getOperationTracker()) {
+ List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
+ if (diskComponents.isEmpty()) {
+ LOGGER.log(Level.INFO, "There are no disk components");
+ return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+ }
+ if (deletedComponents.contains(diskComponents.get(diskComponents.size() - 1))) {
+ LOGGER.log(Level.INFO, "All disk components have been deleted");
+ return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
+ }
+ int mostRecentComponentIndex = 0;
+ for (int i = 0; i < diskComponents.size(); i++) {
+ if (!deletedComponents.contains(diskComponents.get(i))) {
+ break;
+ }
+ mostRecentComponentIndex++;
+ }
+ ILSMDiskComponent mostRecentDiskComponent = diskComponents.get(mostRecentComponentIndex);
+ return (LSMComponentId) mostRecentDiskComponent.getId();
+ }
+ }
+
+ private void putLSNIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+ throws HyracksDataException {
+ putLSNIntoMetadata(newComponent, getComponentLSN(oldComponents));
+ }
+
+ private void putLSNIntoMetadata(ILSMDiskComponent newComponent, long lsn) throws HyracksDataException {
+ newComponent.getMetadata().put(KEY_METADATA_FLUSH_LOG_LSN, LongPointable.FACTORY.createPointable(lsn));
+ }
+
+ public static long getTreeIndexLSN(DiskComponentMetadata md) throws HyracksDataException {
+ LongPointable pointable = new LongPointable();
+ IMetadataPageManager metadataPageManager = md.getMetadataPageManager();
+ metadataPageManager.get(metadataPageManager.createMetadataFrame(), KEY_METADATA_FLUSH_LOG_LSN, pointable);
+ return pointable.getLength() == 0 ? INVALID_LSN : pointable.longValue();
+ }
+
+ private ILSMComponentId getMergedComponentId(List<? extends ILSMComponent> mergedComponents)
+ throws HyracksDataException {
+ if (mergedComponents.isEmpty()) {
+ return null;
+ }
+ return LSMComponentIdUtils.union(mergedComponents.get(0).getId(),
+ mergedComponents.get(mergedComponents.size() - 1).getId());
+ }
+
+ private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, List<ILSMDiskComponent> oldComponents)
+ throws HyracksDataException {
+ ILSMComponentId componentId = getMergedComponentId(oldComponents);
+ putComponentIdIntoMetadata(newComponent, componentId);
+ }
+
+ private void putComponentIdIntoMetadata(ILSMDiskComponent newComponent, ILSMComponentId componentId)
+ throws HyracksDataException {
+ LSMComponentIdUtils.persist(componentId, newComponent.getMetadata());
+ }
+
+ public synchronized void setFirstLsnForCurrentMemoryComponent(long firstLsn) {
+ this.firstLsnForCurrentMemoryComponent = firstLsn;
+ if (pendingFlushes == 0) {
+ this.persistenceLsn = firstLsn;
+ }
+ }
+
+ public synchronized long getPersistenceLsn() {
+ return persistenceLsn;
+ }
+
+ public long getComponentLSN(List<ILSMDiskComponent> diskComponents) throws HyracksDataException {
+ if (diskComponents.isEmpty()) {
+ throw new IllegalArgumentException("Can't get LSN from an empty list of disk components");
+ }
+ // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
+ long maxLSN = -1L;
+ for (ILSMDiskComponent c : diskComponents) {
+ DiskComponentMetadata md = c.getMetadata();
+ maxLSN = Math.max(getTreeIndexLSN(md), maxLSN);
+ }
+ return maxLSN;
+ }
+
+ @Override
+ public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+ component.resetId(componentIds.poll(), false);
+ }
+
+ @Override
+ public synchronized void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+ dsInfo.declareActiveIOOperation();
+ if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ pendingFlushes++;
+ FlushOperation flush = (FlushOperation) operation;
+ Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+ Long flushLsn = (Long) map.get(KEY_FLUSH_LOG_LSN);
+ map.put(KEY_FIRST_LSN, firstLsnForCurrentMemoryComponent);
+ map.put(KEY_FLUSHED_COMPONENT_ID, flush.getFlushingComponent().getId());
+ componentIds.add((ILSMComponentId) map.get(KEY_NEXT_COMPONENT_ID));
+ firstLsnForCurrentMemoryComponent = flushLsn; // Advance the first lsn for new component
+ }
+ }
+
+ @Override
+ public synchronized void completed(ILSMIOOperation operation) {
+ if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ pendingFlushes--;
+ if (operation.getStatus() == LSMIOOperationStatus.SUCCESS) {
+ Map<String, Object> map = operation.getAccessor().getOpContext().getParameters();
+ persistenceLsn =
+ pendingFlushes == 0 ? firstLsnForCurrentMemoryComponent : (Long) map.get(KEY_FLUSH_LOG_LSN);
+ }
+ }
+ dsInfo.undeclareActiveIOOperation();
+ }
+
+ public synchronized boolean hasPendingFlush() {
+ return pendingFlushes > 0;
+ }
+
+ @Override
+ public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+ // No Op
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..826ca50
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMIndexIOOperationCallbackFactory.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.ioopcallbacks;
+
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class LSMIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private final ILSMComponentIdGeneratorFactory idGeneratorFactory;
+
+ protected final IDatasetInfoProvider datasetInfoProvider;
+
+ protected transient INCServiceContext ncCtx;
+
+ protected transient IResource resource;
+
+ public LSMIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory,
+ IDatasetInfoProvider datasetInfoProvider) {
+ this.idGeneratorFactory = idGeneratorFactory;
+ this.datasetInfoProvider = datasetInfoProvider;
+ }
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) {
+ this.ncCtx = ncCtx;
+ this.resource = resource;
+ }
+
+ protected ILSMComponentIdGenerator getComponentIdGenerator() throws HyracksDataException {
+ return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource);
+ }
+
+ protected IIndexCheckpointManagerProvider getIndexCheckpointManagerProvider() {
+ return ((INcApplicationContext) ncCtx.getApplicationContext()).getIndexCheckpointManagerProvider();
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+ return new LSMIOOperationCallback(datasetInfoProvider.getDatasetInfo(ncCtx), index,
+ getComponentIdGenerator().getId(), getIndexCheckpointManagerProvider());
+ }
+
+ @Override
+ public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+ return idGeneratorFactory.getComponentIdGenerator(ncCtx, resource).getCurrentComponentIndex();
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
deleted file mode 100644
index 3ba9bcd..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ /dev/null
@@ -1,32 +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.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
-
- public LSMInvertedIndexIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider checkpointManagerProvider) {
- super(index, idGenerator, checkpointManagerProvider);
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
deleted file mode 100644
index 766ef95..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,40 +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.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMInvertedIndexIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- public LSMInvertedIndexIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- super(idGeneratorFactory);
- }
-
- @Override
- public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
- return new LSMInvertedIndexIOOperationCallback(index, getComponentIdGenerator(),
- getIndexCheckpointManagerProvider());
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
deleted file mode 100644
index f3e80ec..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ /dev/null
@@ -1,32 +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.common.ioopcallbacks;
-
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
-
- public LSMRTreeIOOperationCallback(ILSMIndex index, ILSMComponentIdGenerator idGenerator,
- IIndexCheckpointManagerProvider checkpointManagerProvodier) {
- super(index, idGenerator, checkpointManagerProvodier);
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
deleted file mode 100644
index 3a0afa8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ /dev/null
@@ -1,39 +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.common.ioopcallbacks;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-
-public class LSMRTreeIOOperationCallbackFactory extends AbstractLSMIndexIOOperationCallbackFactory {
-
- private static final long serialVersionUID = 1L;
-
- public LSMRTreeIOOperationCallbackFactory(ILSMComponentIdGeneratorFactory idGeneratorFactory) {
- super(idGeneratorFactory);
- }
-
- @Override
- public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
- return new LSMRTreeIOOperationCallback(index, getComponentIdGenerator(), getIndexCheckpointManagerProvider());
- }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
index b008f11..2c0872c 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IIndexCheckpointManager.java
@@ -27,31 +27,33 @@
/**
* Initializes the first checkpoint of an index with low watermark {@code lsn}
*
+ * @param componentTimestamp
* @param lsn
* @throws HyracksDataException
*/
- void init(long lsn) throws HyracksDataException;
+ void init(String componentTimestamp, long lsn) throws HyracksDataException;
/**
- * Called when a new LSM disk component is flushed. When called, the index checkpoiint is updated
+ * Called when a new LSM disk component is flushed. When called, the index checkpoint is updated
* with the latest valid {@code componentTimestamp} and low watermark {@code lsn}
*
* @param componentTimestamp
* @param lsn
* @throws HyracksDataException
*/
- void flushed(String componentTimestamp, long lsn) throws HyracksDataException;
+ void flushed(String componentTimestamp, long lsn, long componentId) throws HyracksDataException;
/**
- * Called when a new LSM disk component is replicated from master. When called, the index checkpoiint is updated
+ * Called when a new LSM disk component is replicated from master. When called, the index checkpoint is updated
* with the latest valid {@code componentTimestamp} and the local lsn mapping of {@code masterLsn} is set as the
* new low watermark.
*
* @param componentTimestamp
* @param masterLsn
+ * @param componentId
* @throws HyracksDataException
*/
- void replicated(String componentTimestamp, long masterLsn) throws HyracksDataException;
+ void replicated(String componentTimestamp, long masterLsn, long componentId) throws HyracksDataException;
/**
* Called when a flush log is received and replicated from master. The mapping between
@@ -89,13 +91,37 @@
* Gets the index last valid component timestamp if the index has any components. Otherwise {@link Optional#empty()}
*
* @return the index last valid component timestamp
+ * @throws HyracksDataException
*/
- Optional<String> getValidComponentTimestamp();
+ Optional<String> getValidComponentTimestamp() throws HyracksDataException;
/**
* Gets the number of valid checkpoints the index has.
*
* @return the number of valid checkpoints
+ * @throws HyracksDataException
*/
- int getCheckpointCount();
+ int getCheckpointCount() throws HyracksDataException;
+
+ /**
+ * @return the latest checkpoint
+ * @throws HyracksDataException
+ */
+ IndexCheckpoint getLatest() throws HyracksDataException;
+
+ /**
+ * Advance the last valid component timestamp. Used for replicated bulkloaded components
+ *
+ * @param timeStamp
+ * @throws HyracksDataException
+ */
+ void advanceValidComponentTimestamp(String timeStamp) throws HyracksDataException;
+
+ /**
+ * Set the last component id. Used during recovery or after component delete
+ *
+ * @param componentId
+ * @throws HyracksDataException
+ */
+ void setLastComponentId(long componentId) throws HyracksDataException;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index 6e845e1..73d3122 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -23,6 +23,7 @@
import java.util.Map;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.core.JsonProcessingException;
@@ -35,24 +36,28 @@
private long id;
private String validComponentTimestamp;
private long lowWatermark;
+ private long lastComponentId;
private Map<Long, Long> masterNodeFlushMap;
- public static IndexCheckpoint first(long lowWatermark) {
+ public static IndexCheckpoint first(String lastComponentTimestamp, long lowWatermark) {
IndexCheckpoint firstCheckpoint = new IndexCheckpoint();
firstCheckpoint.id = INITIAL_CHECKPOINT_ID;
firstCheckpoint.lowWatermark = lowWatermark;
- firstCheckpoint.validComponentTimestamp = null;
+ firstCheckpoint.validComponentTimestamp = lastComponentTimestamp;
+ firstCheckpoint.lastComponentId = LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID.getMaxId();
firstCheckpoint.masterNodeFlushMap = new HashMap<>();
return firstCheckpoint;
}
- public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, String validComponentTimestamp) {
+ public static IndexCheckpoint next(IndexCheckpoint latest, long lowWatermark, String validComponentTimestamp,
+ long lastComponentId) {
if (lowWatermark < latest.getLowWatermark()) {
throw new IllegalStateException("Low watermark should always be increasing");
}
IndexCheckpoint next = new IndexCheckpoint();
next.id = latest.getId() + 1;
next.lowWatermark = lowWatermark;
+ next.lastComponentId = lastComponentId;
next.validComponentTimestamp = validComponentTimestamp;
next.masterNodeFlushMap = latest.getMasterNodeFlushMap();
// remove any lsn from the map that wont be used anymore
@@ -72,6 +77,10 @@
return lowWatermark;
}
+ public long getLastComponentId() {
+ return lastComponentId;
+ }
+
public Map<Long, Long> getMasterNodeFlushMap() {
return masterNodeFlushMap;
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
deleted file mode 100644
index 82bfdfa..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockHashTable.java
+++ /dev/null
@@ -1,39 +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.common.transactions;
-
-/**
- * @author pouria Interface for a hashTable, used in the internal data
- * structures of lockManager
- * @param <K>
- * Type of the objects, used as keys
- * @param <V>
- * Type of the objects, used as values
- */
-public interface ILockHashTable<K, V> {
-
- public void put(K key, V value);
-
- public V get(K key);
-
- public V remove(K key);
-
- public int getKeysetSize();
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
deleted file mode 100644
index a545ac2..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockMatrix.java
+++ /dev/null
@@ -1,48 +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.common.transactions;
-
-/**
- * @author pouria
- * Shows: - The conflict matrix for the locking protocol (whether two
- * lock modes conflict with each other or not on a single resource) -
- * Whether request to convert a lock mode to a new one is a conversion
- * (i.e. the new lock mode is stringer than the current one) or not
- * Each lock mode is shown/interpreted as an integer
- */
-
-public interface ILockMatrix {
-
- /**
- * @param mask
- * (current/expected) lock mask on the resource
- * @param reqLockMode
- * index of the requested lockMode
- * @return true if the lock request conflicts with the mask
- */
- public boolean conflicts(int mask, int reqLockMode);
-
- /**
- * @param currentLockMode
- * @param reqLockMode
- * @return true if the request is a conversion
- */
- public boolean isConversion(int currentLockMode, int reqLockMode);
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
deleted file mode 100644
index b5ee0a8..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILongBlockFactory.java
+++ /dev/null
@@ -1,40 +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.common.transactions;
-
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public interface ILongBlockFactory {
- /**
- * Ensures future blocks are allocated larger than the supplied value
- *
- * @param value
- * the value to ensure future blocks are larger than
- */
- void ensureMinimum(long value) throws AlgebricksException;
-
- /**
- * Allocates a block of longs of specified block size
- *
- * @param blockSize
- * The size of the block of longs to reserve
- * @return the start of the reserved block
- */
- long getBlock(int blockSize) throws AlgebricksException;
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.java
new file mode 100644
index 0000000..1da5c9c
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManagerFactory.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.asterix.common.transactions;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+
+public interface IRecoveryManagerFactory {
+
+ /**
+ * Create the local recovery manager
+ *
+ * @param serviceCtx
+ * the service context
+ * @param txnSubsystem
+ * the transaction subsystem
+ * @return the recovery manager
+ */
+ IRecoveryManager createRecoveryManager(INCServiceContext serviceCtx, ITransactionSubsystem txnSubsystem);
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
deleted file mode 100644
index 94ca848..0000000
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ITxnIdBlockProvider.java
+++ /dev/null
@@ -1,47 +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.common.transactions;
-
-import java.io.Serializable;
-import java.rmi.Remote;
-import java.rmi.RemoteException;
-
-public interface ITxnIdBlockProvider extends Remote, Serializable {
-
- /**
- * Ensures that future transaction blocks will be of a value larger than the supplied value
- *
- * @param maxId
- * The txn id to ensure future txn ids are larger than
- * @throws RemoteException
- */
- void ensureMinimumTxnId(long maxId) throws RemoteException;
-
- /**
- * Allocates a block of transaction ids of specified block size
- *
- * @param blockSize
- * The size of the transaction id block to reserve
- * @return the start of the reserved block
- * @throws RemoteException
- */
- long reserveTxnIdBlock(int blockSize) throws RemoteException;
-
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 5fdb4e2..0c3b21d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -24,7 +24,6 @@
import java.util.zip.CRC32;
import org.apache.asterix.common.context.PrimaryIndexOperationTracker;
-import org.apache.commons.lang3.ArrayUtils;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleReference;
import org.apache.hyracks.storage.am.common.tuples.SimpleTupleWriter;
@@ -271,6 +270,7 @@
computeAndSetLogSize();
break;
case LogType.WAIT:
+ case LogType.WAIT_FOR_FLUSHES:
computeAndSetLogSize();
break;
case LogType.JOB_COMMIT:
@@ -462,6 +462,7 @@
logSize = FLUSH_LOG_SIZE;
break;
case LogType.WAIT:
+ case LogType.WAIT_FOR_FLUSHES:
logSize = WAIT_LOG_SIZE;
break;
case LogType.FILTER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
index f02b0de..2d76a11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
@@ -28,6 +28,7 @@
public static final byte WAIT = 6;
public static final byte FILTER = 7;
public static final byte MARKER = 8;
+ public static final byte WAIT_FOR_FLUSHES = 9;
private static final String STRING_UPDATE = "UPDATE";
private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
@@ -37,6 +38,7 @@
private static final String STRING_WAIT = "WAIT";
private static final String STRING_FILTER = "FILTER";
private static final String STRING_MARKER = "MARKER";
+ private static final String STRING_WAIT_FOR_FLUSHES = "WAIT_FOR_FLUSHES";
private static final String STRING_UNKNOWN_LOG_TYPE = "UNKNOWN_LOG_TYPE";
public static String toString(byte logType) {
@@ -53,6 +55,8 @@
return STRING_FLUSH;
case LogType.WAIT:
return STRING_WAIT;
+ case LogType.WAIT_FOR_FLUSHES:
+ return STRING_WAIT_FOR_FLUSHES;
case LogType.FILTER:
return STRING_FILTER;
case LogType.MARKER:
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
index 1ac3ffa..b135c7f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/Servlets.java
@@ -20,14 +20,6 @@
public class Servlets {
- public static final String AQL = "/aql";
- public static final String AQL_QUERY = "/query";
- public static final String AQL_UPDATE = "/update";
- public static final String AQL_DDL = "/ddl";
- public static final String SQLPP = "/sqlpp";
- public static final String SQLPP_QUERY = "/query/sqlpp";
- public static final String SQLPP_UPDATE = "/update/sqlpp";
- public static final String SQLPP_DDL = "/ddl/sqlpp";
public static final String QUERY_STATUS = "/query/service/status/*";
public static final String QUERY_RESULT = "/query/service/result/*";
public static final String QUERY_SERVICE = "/query/service";
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
index 265c9fd..4bde9cf 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StorageConstants.java
@@ -37,23 +37,17 @@
public static final String METADATA_FILE_NAME = ".metadata";
public static final String MASK_FILE_PREFIX = ".mask_";
public static final String COMPONENT_MASK_FILE_PREFIX = MASK_FILE_PREFIX + "C_";
- public static final String LEGACY_DATASET_INDEX_NAME_SEPARATOR = "_idx_";
/**
* The storage version of AsterixDB related artifacts (e.g. log files, checkpoint files, etc..).
*/
- private static final int LOCAL_STORAGE_VERSION = 2;
+ private static final int LOCAL_STORAGE_VERSION = 3;
/**
* The storage version of AsterixDB stack.
*/
public static final int VERSION = LOCAL_STORAGE_VERSION + ITreeIndexFrame.Constants.VERSION;
- /**
- * The storage version in which the rebalance storage structure was introduced
- */
- public static final int REBALANCE_STORAGE_VERSION = 8;
-
private StorageConstants() {
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
index 6b13468..aa2c7af 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/utils/StoragePathUtil.java
@@ -19,17 +19,17 @@
package org.apache.asterix.common.utils;
import java.io.File;
+import java.nio.file.Path;
import java.nio.file.Paths;
-import java.util.function.Function;
import org.apache.asterix.common.cluster.ClusterPartition;
-import org.apache.asterix.common.storage.IndexPathElements;
import org.apache.asterix.common.storage.ResourceReference;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileSplit;
+import org.apache.hyracks.api.io.IIOManager;
import org.apache.hyracks.api.io.MappedFileSplit;
import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import org.apache.hyracks.dataflow.std.file.IFileSplitProvider;
@@ -143,4 +143,16 @@
public static String getIndexNameFromPath(String path) {
return Paths.get(path).getFileName().toString();
}
+
+ /**
+ * Get the path of the index containing the passed reference
+ *
+ * @param ioManager
+ * @param ref
+ * @return
+ * @throws HyracksDataException
+ */
+ public static Path getIndexPath(IIOManager ioManager, ResourceReference ref) throws HyracksDataException {
+ return ioManager.resolve(ref.getRelativePath().toString()).getFile().toPath();
+ }
}
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 a0cebea..d62ffde 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -211,7 +211,7 @@
3071 = Found END_RECORD while expecting a list item
3072 = Cannot cast the %1$s type to the %2$s type
3073 = Missing deserializer method for constructor: %1$s
-3074 = This can not be an instance of %1$s
+3074 = %1$s cannot be an instance of %2$s
3075 = Closed field %1$s has null value
3076 = %1$s: no files found
3077 = %1$s: path not found
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index befbeed..843cb21 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -34,7 +34,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -220,7 +219,7 @@
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
- List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+ List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
mergedComponents.forEach(component -> {
try {
resultComponentIDs.add(component.getId());
@@ -230,8 +229,7 @@
});
return null;
}
- }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
- Mockito.anyListOf(ILSMDiskComponent.class));
+ }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
Mockito.when(index.isPrimaryIndex()).thenReturn(isPrimary);
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
deleted file mode 100644
index c03af40..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/AbstractLSMIOOperationCallbackTest.java
+++ /dev/null
@@ -1,315 +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.test.ioopcallbacks;
-
-import java.util.Collections;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.asterix.common.storage.IIndexCheckpointManager;
-import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.Mockito;
-
-import junit.framework.TestCase;
-
-public abstract class AbstractLSMIOOperationCallbackTest extends TestCase {
-
- @Test
- public void testNormalSequence() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
- LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
-
- //request to flush first component
- callback.updateLastLSN(1);
- firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(firstOpCtx);
-
- ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
- //request to flush second component
- callback.updateLastLSN(2);
- secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(secondOpCtx);
-
- Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
- final ILSMDiskComponent diskComponent1 = mockDiskComponent();
- firstOpCtx.setNewComponent(diskComponent1);
- callback.afterOperation(firstOpCtx);
- callback.afterFinalize(firstOpCtx);
-
- Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
- final ILSMDiskComponent diskComponent2 = mockDiskComponent();
- secondOpCtx.setNewComponent(diskComponent2);
- callback.afterOperation(secondOpCtx);
- callback.afterFinalize(secondOpCtx);
- }
-
- @Test
- public void testOverWrittenLSN() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
- LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
-
- //request to flush first component
- ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
- callback.updateLastLSN(1);
- firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(firstOpCtx);
-
- //request to flush second component
- ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
- callback.updateLastLSN(2);
- secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(secondOpCtx);
-
- //request to flush first component again
- //this call should fail
- callback.updateLastLSN(3);
- //there is no corresponding beforeOperation, since the first component is being flush
- //the scheduleFlush request would fail this time
-
- Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
- final ILSMDiskComponent diskComponent1 = mockDiskComponent();
- firstOpCtx.setNewComponent(diskComponent1);
- callback.afterOperation(firstOpCtx);
- callback.afterFinalize(firstOpCtx);
- final ILSMDiskComponent diskComponent2 = mockDiskComponent();
- secondOpCtx.setNewComponent(diskComponent2);
- Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
- callback.afterOperation(secondOpCtx);
- callback.afterFinalize(secondOpCtx);
- }
-
- @Test
- public void testLostLSN() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
-
- LSMBTreeIOOperationCallback callback = new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- //request to flush first component
- ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
- callback.updateLastLSN(1);
- firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(firstOpCtx);
-
- //request to flush second component
- ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
- callback.updateLastLSN(2);
- secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(secondOpCtx);
-
- Assert.assertEquals(1, callback.getComponentLSN(Collections.emptyList()));
-
- // the first flush is finished, but has not finalized yet (in codebase, these two calls
- // are not synchronized)
- firstOpCtx.setNewComponent(mockDiskComponent());
- callback.afterOperation(firstOpCtx);
-
- //request to flush first component again
- callback.updateLastLSN(3);
-
- // the first flush is finalized (it may be called after afterOperation for a while)
- callback.afterFinalize(firstOpCtx);
-
- // the second flush gets LSN 2
- Assert.assertEquals(2, callback.getComponentLSN(Collections.emptyList()));
- // the second flush is finished
- secondOpCtx.setNewComponent(mockDiskComponent());
- callback.afterOperation(secondOpCtx);
- callback.afterFinalize(secondOpCtx);
-
- // it should get new LSN 3
- Assert.assertEquals(3, callback.getComponentLSN(Collections.emptyList()));
- }
-
- @Test
- public void testAllocateComponentId() throws HyracksDataException {
- ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
-
- LSMBTreeIOOperationCallback callback =
- new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
- ILSMComponentId initialId = idGenerator.getId();
- // simulate a partition is flushed before allocated
- idGenerator.refresh();
- callback.updateLastLSN(0);
-
- callback.allocated(mockComponent);
- checkMemoryComponent(initialId, mockComponent);
- }
-
- @Test
- public void testRecycleComponentId() throws HyracksDataException {
- ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
- LSMBTreeIOOperationCallback callback =
- new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
- ILSMComponentId id = idGenerator.getId();
- callback.allocated(mockComponent);
- checkMemoryComponent(id, mockComponent);
-
- Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
- for (int i = 0; i < 100; i++) {
- // schedule a flush
- idGenerator.refresh();
- ILSMComponentId expectedId = idGenerator.getId();
- callback.updateLastLSN(0);
- ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
- opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(opCtx);
- callback.recycled(mockComponent, true);
- opCtx.setNewComponent(mockDiskComponent());
- callback.afterOperation(opCtx);
- callback.afterFinalize(opCtx);
- checkMemoryComponent(expectedId, mockComponent);
- }
- }
-
- @Test
- public void testRecycleWithoutSwitch() throws HyracksDataException {
- ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
- LSMBTreeIOOperationCallback callback =
- new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
- ILSMComponentId id = idGenerator.getId();
- callback.allocated(mockComponent);
- checkMemoryComponent(id, mockComponent);
- Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
- for (int i = 0; i < 10; i++) {
- idGenerator.refresh();
- id = idGenerator.getId();
- callback.updateLastLSN(0);
- // Huh! There is no beforeOperation?
- ILSMIndexOperationContext opCtx = new TestLSMIndexOperationContext(mockIndex);
- opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.recycled(mockComponent, false);
- callback.afterFinalize(opCtx);
- checkMemoryComponent(id, mockComponent);
- }
- }
-
- @Test
- public void testConcurrentRecycleComponentId() throws HyracksDataException {
- ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator();
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
- Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- LSMBTreeIOOperationCallback callback =
- new LSMBTreeIOOperationCallback(mockIndex, idGenerator, mockIndexCheckpointManagerProvider());
-
- ILSMComponentId id = idGenerator.getId();
- callback.allocated(mockComponent);
- checkMemoryComponent(id, mockComponent);
-
- Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
-
- // schedule a flush
- idGenerator.refresh();
- ILSMComponentId expectedId = idGenerator.getId();
-
- callback.updateLastLSN(0);
- ILSMIndexOperationContext firstOpCtx = new TestLSMIndexOperationContext(mockIndex);
- firstOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(firstOpCtx);
- firstOpCtx.setNewComponent(mockDiskComponent());
- callback.afterOperation(firstOpCtx);
- callback.afterFinalize(firstOpCtx);
-
- // another flush is to be scheduled before the component is recycled
- idGenerator.refresh();
- ILSMComponentId nextId = idGenerator.getId();
-
- // recycle the component
- callback.recycled(mockComponent, true);
- checkMemoryComponent(expectedId, mockComponent);
-
- // schedule the next flush
- callback.updateLastLSN(0);
- ILSMIndexOperationContext secondOpCtx = new TestLSMIndexOperationContext(mockIndex);
- secondOpCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.beforeOperation(secondOpCtx);
- secondOpCtx.setNewComponent(mockDiskComponent());
- callback.afterOperation(secondOpCtx);
- callback.afterFinalize(secondOpCtx);
- callback.recycled(mockComponent, true);
- checkMemoryComponent(nextId, mockComponent);
- }
-
- private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
- throws HyracksDataException {
- ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
- ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
- Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
- assertEquals(expected, idArgument.getValue());
- assertEquals(false, forceArgument.getValue().booleanValue());
-
- Mockito.reset(memoryComponent);
- }
-
- private ILSMDiskComponent mockDiskComponent() {
- ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
- Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
- return component;
- }
-
- protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
- IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
- Mockito.mock(IIndexCheckpointManagerProvider.class);
- IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
- Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong());
- Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
- return indexCheckpointManagerProvider;
- }
-
- protected abstract AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException;
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
deleted file mode 100644
index a4bc399..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +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.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
- @Override
- protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- return new LSMBTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
deleted file mode 100644
index 5f37c78..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMBTreeWithBuddyIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +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.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMBTreeWithBuddyIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
- @Override
- protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- return new LSMBTreeWithBuddyIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
new file mode 100644
index 0000000..29a2aa0
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMIOOperationCallbackTest.java
@@ -0,0 +1,230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.ioopcallbacks;
+
+import java.text.Format;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.context.DatasetInfo;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
+import org.apache.asterix.common.storage.IIndexCheckpointManager;
+import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.DiskComponentMetadata;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+
+import junit.framework.TestCase;
+
+public class LSMIOOperationCallbackTest extends TestCase {
+ /*
+ * The normal sequence of calls:
+ * 1. refresh id generator
+ * 2. flushLsn
+ * 3. created
+ * 4. before
+ * 5. after
+ * 6. finalize
+ * 7. destroy
+ */
+
+ private static final Format FORMATTER =
+ new SimpleDateFormat(AbstractLSMIndexFileManager.COMPONENT_TIMESTAMP_FORMAT);
+
+ private static String getComponentFileName() {
+ Date date = new Date();
+ String ts = FORMATTER.format(date);
+ return ts + '_' + ts;
+ }
+
+ @Test
+ public void testNormalSequence() throws HyracksDataException {
+ int numMemoryComponents = 2;
+
+ ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+ String indexId = "mockIndexId";
+ Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+ Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(Mockito.mock(AbstractLSMMemoryComponent.class));
+ DatasetInfo dsInfo = new DatasetInfo(101, null);
+ LSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+ LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+ mockIndexCheckpointManagerProvider());
+ //Flush first
+ idGenerator.refresh();
+ long flushLsn = 1L;
+ ILSMComponentId nextComponentId = idGenerator.getId();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+ ILSMIndexAccessor firstAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+ firstAccessor.getOpContext().setParameters(flushMap);
+ FileReference firstTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+ LSMComponentFileReferences firstFiles = new LSMComponentFileReferences(firstTarget, firstTarget, firstTarget);
+ FlushOperation firstFlush = new TestFlushOperation(firstAccessor, firstTarget, callback, indexId, firstFiles,
+ new LSMComponentId(0, 0));
+ callback.scheduled(firstFlush);
+ callback.beforeOperation(firstFlush);
+
+ //Flush second
+ idGenerator.refresh();
+ flushLsn = 2L;
+ nextComponentId = idGenerator.getId();
+ flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+ ILSMIndexAccessor secondAccessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+ secondAccessor.getOpContext().setParameters(flushMap);
+ FileReference secondTarget = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+ LSMComponentFileReferences secondFiles =
+ new LSMComponentFileReferences(secondTarget, secondTarget, secondTarget);
+ FlushOperation secondFlush = new TestFlushOperation(secondAccessor, secondTarget, callback, indexId,
+ secondFiles, new LSMComponentId(1, 1));
+ callback.scheduled(secondFlush);
+ callback.beforeOperation(secondFlush);
+
+ Map<String, Object> firstFlushMap = firstFlush.getAccessor().getOpContext().getParameters();
+ long firstFlushLogLsn = (Long) firstFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+ Assert.assertEquals(1, firstFlushLogLsn);
+ final ILSMDiskComponent diskComponent1 = mockDiskComponent();
+ firstFlush.setNewComponent(diskComponent1);
+ callback.afterOperation(firstFlush);
+ callback.afterFinalize(firstFlush);
+ callback.completed(firstFlush);
+
+ Map<String, Object> secondFlushMap = secondFlush.getAccessor().getOpContext().getParameters();
+ long secondFlushLogLsn = (Long) secondFlushMap.get(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN);
+ Assert.assertEquals(2, secondFlushLogLsn);
+ final ILSMDiskComponent diskComponent2 = mockDiskComponent();
+ secondFlush.setNewComponent(diskComponent2);
+ callback.afterOperation(secondFlush);
+ callback.afterFinalize(secondFlush);
+ callback.completed(secondFlush);
+ }
+
+ @Test
+ public void testAllocateComponentId() throws HyracksDataException {
+ int numMemoryComponents = 2;
+ DatasetInfo dsInfo = new DatasetInfo(101, null);
+ ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+ ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+ Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+ ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+ Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+ LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+ mockIndexCheckpointManagerProvider());
+ ILSMComponentId initialId = idGenerator.getId();
+ // simulate a partition is flushed before allocated
+ idGenerator.refresh();
+ long flushLsn = 1L;
+ ILSMComponentId nextComponentId = idGenerator.getId();
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, nextComponentId);
+ callback.allocated(mockComponent);
+ callback.recycled(mockComponent);
+ checkMemoryComponent(initialId, mockComponent);
+ }
+
+ @Test
+ public void testRecycleComponentId() throws HyracksDataException {
+ int numMemoryComponents = 2;
+ DatasetInfo dsInfo = new DatasetInfo(101, null);
+ ILSMComponentIdGenerator idGenerator = new LSMComponentIdGenerator(numMemoryComponents);
+ ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
+ Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(numMemoryComponents);
+ ILSMMemoryComponent mockComponent = Mockito.mock(AbstractLSMMemoryComponent.class);
+ Mockito.when(mockIndex.getCurrentMemoryComponent()).thenReturn(mockComponent);
+ LSMIOOperationCallback callback = new LSMIOOperationCallback(dsInfo, mockIndex, idGenerator.getId(),
+ mockIndexCheckpointManagerProvider());
+ String indexId = "mockIndexId";
+ ILSMComponentId id = idGenerator.getId();
+ callback.recycled(mockComponent);
+ checkMemoryComponent(id, mockComponent);
+
+ Mockito.when(mockIndex.isMemoryComponentsAllocated()).thenReturn(true);
+ for (int i = 0; i < 100; i++) {
+ // schedule a flush
+ idGenerator.refresh();
+ ILSMComponentId expectedId = idGenerator.getId();
+ long flushLsn = 0L;
+ Map<String, Object> flushMap = new HashMap<>();
+ flushMap.put(LSMIOOperationCallback.KEY_FLUSH_LOG_LSN, flushLsn);
+ flushMap.put(LSMIOOperationCallback.KEY_NEXT_COMPONENT_ID, expectedId);
+ ILSMIndexAccessor accessor = new TestLSMIndexAccessor(new TestLSMIndexOperationContext(mockIndex));
+ accessor.getOpContext().setParameters(flushMap);
+ FileReference target = new FileReference(Mockito.mock(IODeviceHandle.class), getComponentFileName());
+ LSMComponentFileReferences files = new LSMComponentFileReferences(target, target, target);
+ FlushOperation flush =
+ new TestFlushOperation(accessor, target, callback, indexId, files, new LSMComponentId(0, 0));
+ callback.scheduled(flush);
+ callback.beforeOperation(flush);
+ callback.recycled(mockComponent);
+ flush.setNewComponent(mockDiskComponent());
+ callback.afterOperation(flush);
+ callback.afterFinalize(flush);
+ callback.completed(flush);
+ checkMemoryComponent(expectedId, mockComponent);
+ }
+ }
+
+ private void checkMemoryComponent(ILSMComponentId expected, ILSMMemoryComponent memoryComponent)
+ throws HyracksDataException {
+ ArgumentCaptor<ILSMComponentId> idArgument = ArgumentCaptor.forClass(ILSMComponentId.class);
+ ArgumentCaptor<Boolean> forceArgument = ArgumentCaptor.forClass(Boolean.class);
+ Mockito.verify(memoryComponent).resetId(idArgument.capture(), forceArgument.capture());
+ assertEquals(expected, idArgument.getValue());
+ assertEquals(false, forceArgument.getValue().booleanValue());
+ Mockito.reset(memoryComponent);
+ }
+
+ private ILSMDiskComponent mockDiskComponent() {
+ ILSMDiskComponent component = Mockito.mock(ILSMDiskComponent.class);
+ Mockito.when(component.getMetadata()).thenReturn(Mockito.mock(DiskComponentMetadata.class));
+ return component;
+ }
+
+ protected IIndexCheckpointManagerProvider mockIndexCheckpointManagerProvider() throws HyracksDataException {
+ IIndexCheckpointManagerProvider indexCheckpointManagerProvider =
+ Mockito.mock(IIndexCheckpointManagerProvider.class);
+ IIndexCheckpointManager indexCheckpointManager = Mockito.mock(IIndexCheckpointManager.class);
+ Mockito.doNothing().when(indexCheckpointManager).flushed(Mockito.any(), Mockito.anyLong(), Mockito.anyLong());
+ Mockito.doReturn(indexCheckpointManager).when(indexCheckpointManagerProvider).get(Mockito.any());
+ return indexCheckpointManagerProvider;
+ }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
deleted file mode 100644
index 343bc59..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMInvertedIndexIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +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.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMInvertedIndexIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
- @Override
- protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- return new LSMInvertedIndexIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
deleted file mode 100644
index 10d95d8..0000000
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/LSMRTreeIOOperationCallbackTest.java
+++ /dev/null
@@ -1,39 +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.test.ioopcallbacks;
-
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallback;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentIdGenerator;
-import org.mockito.Mockito;
-
-public class LSMRTreeIOOperationCallbackTest extends AbstractLSMIOOperationCallbackTest {
-
- @Override
- protected AbstractLSMIOOperationCallback getIoCallback() throws HyracksDataException {
- ILSMIndex mockIndex = Mockito.mock(ILSMIndex.class);
- Mockito.when(mockIndex.getNumberOfAllMemoryComponents()).thenReturn(2);
- return new LSMRTreeIOOperationCallback(mockIndex, new LSMComponentIdGenerator(),
- mockIndexCheckpointManagerProvider());
- }
-
-}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
new file mode 100644
index 0000000..8a02aa3
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestFlushOperation.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.ioopcallbacks;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.impls.FlushOperation;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.mockito.Mockito;
+
+public class TestFlushOperation extends FlushOperation {
+
+ private final LSMComponentFileReferences files;
+ private final ILSMMemoryComponent flushingComponent;
+
+ public TestFlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
+ String indexIdentifier, LSMComponentFileReferences files, LSMComponentId componentId)
+ throws HyracksDataException {
+ super(accessor, target, callback, indexIdentifier);
+ this.files = files;
+ flushingComponent = accessor.getOpContext().getIndex().getCurrentMemoryComponent();
+ Mockito.when(flushingComponent.getId()).thenReturn(componentId);
+ }
+
+ @Override
+ public LSMComponentFileReferences getComponentFiles() {
+ return files;
+ }
+
+ @Override
+ public ILSMComponent getFlushingComponent() {
+ return flushingComponent;
+ }
+
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
new file mode 100644
index 0000000..c2621d8
--- /dev/null
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexAccessor.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.test.ioopcallbacks;
+
+import java.util.List;
+import java.util.function.Predicate;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+
+public class TestLSMIndexAccessor implements ILSMIndexAccessor {
+
+ private final ILSMIndexOperationContext opCtx;
+
+ public TestLSMIndexAccessor(ILSMIndexOperationContext opCtx) {
+ this.opCtx = opCtx;
+ }
+
+ @Override
+ public void insert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void update(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void delete(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void upsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public IIndexCursor createSearchCursor(boolean exclusive) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void destroy() throws HyracksDataException {
+ }
+
+ @Override
+ public ILSMIndexOperationContext getOpContext() {
+ return opCtx;
+ }
+
+ @Override
+ public ILSMIOOperation scheduleFlush() throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void physicalDelete(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean tryInsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean tryDelete(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean tryUpdate(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean tryUpsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void forcePhysicalDelete(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void forceInsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void forceDelete(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void forceUpsert(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
+ throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void flush(ILSMIOOperation operation) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void merge(ILSMIOOperation operation) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void updateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void forceUpdateMeta(IValueReference key, IValueReference value) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void scanDiskComponents(IIndexCursor cursor) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void updateFilter(ITupleReference tuple) throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
index 9b749fa..4306dda 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/ioopcallbacks/TestLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -28,10 +29,9 @@
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -44,8 +44,8 @@
private final List<ILSMDiskComponent> componentsToBeReplicated = new ArrayList<>();
private boolean isAccessingComponents;
private IndexOperation op;
- private LSMIOOperationType ioOperationType;
- private ILSMDiskComponent newComponent;
+ private ILSMIOOperation ioOperation;
+ private Map<String, Object> map;
private boolean filterSkip = false;
private boolean isRecovery = false;
@@ -180,22 +180,23 @@
}
@Override
- public LSMIOOperationType getIoOperationType() {
- return ioOperationType;
+ public ILSMIOOperation getIoOperation() {
+ return ioOperation;
}
@Override
- public void setIoOperationType(LSMIOOperationType ioOpType) {
- this.ioOperationType = ioOpType;
+ public void setIoOperation(ILSMIOOperation ioOperation) {
+ this.ioOperation = ioOperation;
}
@Override
- public ILSMDiskComponent getNewComponent() {
- return newComponent;
+ public void setParameters(Map<String, Object> map) {
+ this.map = map;
}
@Override
- public void setNewComponent(ILSMDiskComponent component) {
- this.newComponent = component;
+ public Map<String, Object> getParameters() {
+ return map;
}
+
}
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
index ac2a465..dfb1b0b 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/12_misc.md
@@ -76,6 +76,28 @@
{ "v1": false, "v2": true, "v3": null }
+### random ###
+ * Syntax:
+
+ random( [seed_value] )
+
+ * Returns a random number, accepting an optional seed value
+ * Arguments:
+ * `seed_value`: an optional `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value representing the seed number.
+ * Return Value:
+ * A random number of type `double` between 0 and 1,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+ {
+ "v1": random(),
+ "v2": random(unix_time_from_datetime_in_ms(current_datetime()))
+ };
+
+
### range ###
* Syntax:
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
index ed9551e..f0e98fd 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/13_conditional.md
@@ -199,3 +199,150 @@
{ "a": true, "b": true, "c": true, "d": 1, "e": true, "f": true, "g": 1 }
The function has an alias `ifnanorinf`.
+
+
+### null_if (nullif) ###
+
+ * Syntax:
+
+ null_if(expression1, expression2)
+
+ * Compares two arguments and returns `null` if they are equal, otherwise returns the first argument.
+ * Arguments:
+ * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+ * `missing` if any argument is a `missing` value,
+ * `null` if
+ * any argument is a `null` value but no argument is a `missing` value, or
+ * `argument1` = `argument2`
+ * a value of the first argument otherwise
+
+ * Example:
+
+ {
+ "a": null_if("asterixdb", "asterixdb"),
+ "b": null_if(1, 2)
+ };
+
+ * The expected result is:
+
+ { "a": null, "b": 1 }
+
+ The function has an alias `nullif`.
+
+
+### missing_if (missingif) ###
+
+ * Syntax:
+
+ missing_if(expression1, expression2)
+
+ * Compares two arguments and returns `missing` if they are equal, otherwise returns the first argument.
+ * Arguments:
+ * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+ * `missing` if
+ * any argument is a `missing` value, or
+ * no argument is a `null` value and `argument1` = `argument2`
+ * `null` if any argument is a `null` value but no argument is a `missing` value
+ * a value of the first argument otherwise
+
+ * Example:
+
+ {
+ "a": missing_if("asterixdb", "asterixdb")
+ "b": missing_if(1, 2),
+ };
+
+ * The expected result is:
+
+ { "b": 1 }
+
+ The function has an alias `missingif`.
+
+
+### nan_if (nanif) ###
+
+ * Syntax:
+
+ nan_if(expression1, expression2)
+
+ * Compares two arguments and returns `NaN` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+ * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+ * `missing` if any argument is a `missing` value,
+ * `null` if any argument is a `null` value but no argument is a `missing` value
+ * `NaN` value of type `double` if `argument1` = `argument2`
+ * a value of the first argument otherwise
+
+ * Example:
+
+ {
+ "a": to_string(nan_if("asterixdb", "asterixdb")),
+ "b": nan_if(1, 2)
+ };
+
+ * The expected result is:
+
+ { "a": "NaN", "b": 1 }
+
+ The function has an alias `nanif`.
+
+
+### posinf_if (posinfif) ###
+
+ * Syntax:
+
+ posinf_if(expression1, expression2)
+
+ * Compares two arguments and returns `+INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+ * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+ * `missing` if any argument is a `missing` value,
+ * `null` if any argument is a `null` value but no argument is a `missing` value
+ * `+INF` value of type `double` if `argument1` = `argument2`
+ * a value of the first argument otherwise
+
+ * Example:
+
+ {
+ "a": to_string(posinf_if("asterixdb", "asterixdb")),
+ "b": posinf_if(1, 2)
+ };
+
+ * The expected result is:
+
+ { "a": "+INF", "b": 1 }
+
+ The function has an alias `posinfif`.
+
+
+### neginf_if (neginfif) ###
+
+ * Syntax:
+
+ neginf_if(expression1, expression2)
+
+ * Compares two arguments and returns `-INF` value if they are equal, otherwise returns the first argument.
+ * Arguments:
+ * `expressionI` : an expression (any type is allowed).
+ * Return Value:
+ * `missing` if any argument is a `missing` value,
+ * `null` if any argument is a `null` value but no argument is a `missing` value
+ * `-INF` value of type `double` if `argument1` = `argument2`
+ * a value of the first argument otherwise
+
+ * Example:
+
+ {
+ "a": to_string(neginf_if("asterixdb", "asterixdb")),
+ "b": neginf_if(1, 2)
+ };
+
+ * The expected result is:
+
+ { "a": "-INF", "b": 1 }
+
+ The function has an alias `neginfif`.
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
index c058d10..b95e778c 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/1_numeric_common.md
@@ -198,6 +198,47 @@
{ "v1": 0.5403023058681398, "v2": -0.4161468365471424, "v3": 1.0, "v4": 0.8775825618903728, "v5": 0.562379076290703 }
+### degrees ###
+ * Syntax:
+
+ degrees(numeric_value)
+
+ * Converts radians to degrees
+ * Arguments:
+ * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+ * The degrees value for the given radians value. The returned value has type `double`,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+ { "v1": degrees(pi()) };
+
+
+ * The expected result is:
+
+ { "v1": 180.0 }
+
+
+### e ###
+ * Syntax:
+
+ e()
+
+ * Return Value:
+ * e (base of the natural logarithm)
+
+ * Example:
+
+ { "v1": e() };
+
+ * The expected result is:
+
+ { "v1": 2.718281828459045 }
+
+
### exp ###
* Syntax:
@@ -301,6 +342,22 @@
{ "v1": 0.0, "v2": 0.3010299956639812, "v3": "-Infinity", "v4": -0.3010299956639812, "v5": 3.0 }
+### pi ###
+ * Syntax:
+
+ pi()
+
+ * Return Value:
+ * Pi
+
+ * Example:
+
+ { "v1": pi() };
+
+ * The expected result is:
+
+ { "v1": 3.141592653589793 }
+
### power ###
* Syntax:
@@ -327,6 +384,30 @@
{ "v1": 1, "v3": 0, "v4": 1.4142135623730951 }
+### radians ###
+ * Syntax:
+
+ radians(numeric_value)
+
+ * Converts degrees to radians
+ * Arguments:
+ * `numeric_value`: a `tinyint`/`smallint`/`integer`/`bigint`/`float`/`double` value.
+ * Return Value:
+ * The radians value for the given degrees value. The returned value has type `double`,
+ * `missing` if the argument is a `missing` value,
+ * `null` if the argument is a `null` value,
+ * any other non-numeric input value will cause a type error.
+
+ * Example:
+
+ { "v1": radians(180) };
+
+
+ * The expected result is:
+
+ { "v1": 3.141592653589793 }
+
+
### round ###
* Syntax:
diff --git a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
index 0e548a7..6599dc3 100644
--- a/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
+++ b/asterixdb/asterix-doc/src/main/markdown/builtins/2_string_common.md
@@ -414,6 +414,30 @@
"v2": "product-a and x-phone"
}
+### reverse ###
+ * Syntax:
+
+ reverse(string)
+
+ * Returns a string formed by reversing characters in the input `string`.
+ * Arguments:
+ * `string` : a `string` to be reversed
+ * Return Value:
+ * a string containing characters from the the input `string` in the reverse order,
+ * `missing` if any argument is a `missing` value,
+ * `null` if any argument is a `null` value but no argument is a `missing` value,
+ * a type error will be raised if:
+ * the first argument is any other non-string value
+
+ * Example:
+
+ reverse("hello");
+
+
+ * The expected result is:
+
+ "olleh"
+
### rtrim ###
* Syntax:
@@ -504,7 +528,7 @@
* Arguments:
* `string` : a `string` to be extracted,
* `offset` : an `tinyint`/`smallint`/`integer`/`bigint` value as the starting offset of the substring in `string`
- (starting at 0),
+ (starting at 0). If negative then counted from the end of the string,
* `length` : (Optional) an an `tinyint`/`smallint`/`integer`/`bigint` value as the length of the substring.
* Return Value:
* a `string` that represents the substring,
@@ -524,6 +548,7 @@
"str"
+The function has an alias `substring`.
### trim ###
* Syntax:
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
index 84986ae..ebf8908 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
@@ -26,7 +26,6 @@
* [Collection Operators](#Collection_operators)
* [Comparison Operators](#Comparison_operators)
* [Logical Operators](#Logical_operators)
- * [Case Expressions](#Case_expressions)
* [Quantified Expressions](#Quantified_expressions)
* [Path Expressions](#Path_expressions)
* [Primary Expressions](#Primary_expressions)
@@ -34,6 +33,7 @@
* [Variable References](#Variable_references)
* [Parenthesized Expressions](#Parenthesized_expressions)
* [Function call Expressions](#Function_call_expressions)
+ * [Case Expressions](#Case_expressions)
* [Constructors](#Constructors)
* [3. Queries](#Queries)
* [Declarations](#Declarations)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index 5f407ca..8fe379d 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -19,11 +19,10 @@
SQL++ is a highly composable expression language. Each SQL++ expression returns zero or more data model instances.
There are three major kinds of expressions in SQL++. At the topmost level, a SQL++ expression can be an
-OperatorExpression (similar to a mathematical expression), an ConditionalExpression (to choose between
-alternative values), or a QuantifiedExpression (which yields a boolean value). Each will be detailed as we
-explore the full SQL++ grammar.
+OperatorExpression (similar to a mathematical expression), or a QuantifiedExpression (which yields a boolean value).
+Each will be detailed as we explore the full SQL++ grammar.
- Expression ::= OperatorExpression | CaseExpression | QuantifiedExpression
+ Expression ::= OperatorExpression | QuantifiedExpression
Note that in the following text, words enclosed in angle brackets denote keywords that are not case-sensitive.
@@ -50,9 +49,9 @@
|-----------------------------------------------------------------------------|-----------|
| EXISTS, NOT EXISTS | Collection emptiness testing |
| ^ | Exponentiation |
-| *, /, % | Multiplication, division, modulo |
+| *, /, DIV, MOD (%) | Multiplication, division, modulo |
| +, - | Addition, subtraction |
-| || | String concatenation |
+| || | String concatenation |
| IS NULL, IS NOT NULL, IS MISSING, IS NOT MISSING, <br/>IS UNKNOWN, IS NOT UNKNOWN, IS VALUED, IS NOT VALUED | Unknown value comparison |
| BETWEEN, NOT BETWEEN | Range comparison (inclusive on both sides) |
| =, !=, <>, <, >, <=, >=, LIKE, NOT LIKE, IN, NOT IN | Comparison |
@@ -72,7 +71,10 @@
|--------------|-------------------------------------------------------------------------|------------|
| +, - | As unary operators, they denote a <br/>positive or negative expression | SELECT VALUE -1; |
| +, - | As binary operators, they add or subtract | SELECT VALUE 1 + 2; |
-| *, /, % | Multiply, divide, modulo | SELECT VALUE 4 / 2.0; |
+| * | Multiply | SELECT VALUE 4 * 2; |
+| / | Divide (returns a value of type `double` if both operands are integers)| SELECT VALUE 5 / 2; |
+| DIV | Divide (returns an integer value if both operands are integers) | SELECT VALUE 5 DIV 2; |
+| MOD (%) | Modulo | SELECT VALUE 5 % 2; |
| ^ | Exponentiation | SELECT VALUE 2^3; |
| || | String concatenation | SELECT VALUE "ab"||"c"||"d"; |
@@ -103,16 +105,16 @@
The following table enumerates all of SQL++'s comparison operators.
-| Operator | Purpose | Example |
-|----------------|--------------------------------------------|------------|
+| Operator | Purpose | Example |
+|----------------|------------------------------------------------|------------|
| IS NULL | Test if a value is NULL | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NULL; |
| IS NOT NULL | Test if a value is not NULL | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT NULL; |
| IS MISSING | Test if a value is MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS MISSING; |
| IS NOT MISSING | Test if a value is not MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT MISSING;|
| IS UNKNOWN | Test if a value is NULL or MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS UNKNOWN; |
| IS NOT UNKNOWN | Test if a value is neither NULL nor MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT UNKNOWN;|
-| IS VALUED | Test if a value is neither NULL nor MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS VALUED; |
-| IS NOT VALUED | Test if a value is NULL or MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT VALUED;|
+| IS KNOWN (IS VALUED) | Test if a value is neither NULL nor MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS KNOWN; |
+| IS NOT KNOWN (IS NOT VALUED) | Test if a value is NULL or MISSING | SELECT * FROM ChirpMessages cm <br/>WHERE cm.user.name IS NOT KNOWN; |
| BETWEEN | Test if a value is between a start value and <br/>a end value. The comparison is inclusive <br/>to both start and end values. | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId BETWEEN 10 AND 20;|
| = | Equality test | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId=10; |
| != | Inequality test | SELECT * FROM ChirpMessages cm <br/>WHERE cm.chirpId!=10;|
@@ -134,8 +136,8 @@
| IS NOT MISSING | TRUE | TRUE | FALSE |
| IS UNKNOWN | FALSE | TRUE | TRUE |
| IS NOT UNKNOWN | TRUE | FALSE | FALSE|
-| IS VALUED | TRUE | FALSE | FALSE |
-| IS NOT VALUED | FALSE | TRUE | TRUE |
+| IS KNOWN (IS VALUED) | TRUE | FALSE | FALSE |
+| IS NOT KNOWN (IS NOT VALUED) | FALSE | TRUE | TRUE |
### <a id="Logical_operators">Logical Operators</a>
Logical operators perform logical `NOT`, `AND`, and `OR` operations over Boolean values (`TRUE` and `FALSE`) plus `NULL` and `MISSING`.
@@ -170,20 +172,6 @@
| NULL | NULL |
| MISSING | MISSING |
-## <a id="Case_expressions">Case Expressions</a>
-
- CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
- SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
- SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
-
-In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
-
-In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
-
-The following example illustrates the form of a case expression.
-##### Example
-
- CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
## <a id="Quantified_expressions">Quantified Expressions</a>
@@ -241,11 +229,13 @@
| VariableReference
| ParenthesizedExpression
| FunctionCallExpression
+ | CaseExpression
| Constructor
The most basic building block for any SQL++ expression is PrimaryExpression. This can be a simple literal (constant)
-value, a reference to a query variable that is in scope, a parenthesized expression, a function call, or a newly
-constructed instance of the data model (such as a newly constructed object, array, or multiset of data model instances).
+value, a reference to a query variable that is in scope, a parenthesized expression, a function call, a conditional
+(case) expression, or a newly constructed instance of the data model (such as a newly constructed object, array,
+or multiset of data model instances).
## <a id="Literals">Literals</a>
@@ -361,6 +351,22 @@
length('a string')
+## <a id="Case_expressions">Case Expressions</a>
+
+ CaseExpression ::= SimpleCaseExpression | SearchedCaseExpression
+ SimpleCaseExpression ::= <CASE> Expression ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+ SearchedCaseExpression ::= <CASE> ( <WHEN> Expression <THEN> Expression )+ ( <ELSE> Expression )? <END>
+
+In a simple `CASE` expression, the query evaluator searches for the first `WHEN` ... `THEN` pair in which the `WHEN` expression is equal to the expression following `CASE` and returns the expression following `THEN`. If none of the `WHEN` ... `THEN` pairs meet this condition, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, `NULL` is returned.
+
+In a searched CASE expression, the query evaluator searches from left to right until it finds a `WHEN` expression that is evaluated to `TRUE`, and then returns its corresponding `THEN` expression. If no condition is found to be `TRUE`, and an `ELSE` branch exists, it returns the `ELSE` expression. Otherwise, it returns `NULL`.
+
+The following example illustrates the form of a case expression.
+
+##### Example
+
+ CASE (2 < 3) WHEN true THEN "yes" ELSE "no" END
+
### <a id="Constructors">Constructors</a>
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
index bc0b398..1328ed9 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/fulltext.md
@@ -44,73 +44,61 @@
ftcontains(Expression1, Expression2, {FullTextOption})
ftcontains(Expression1, Expression2)
-For example, we can execute the following query to find tweet messages where the `message-text` field includes
+For example, we can execute the following query to find Chirp messages where the `messageText` field includes
“voice” as a word. Please note that an FTS search is case-insensitive.
Thus, "Voice" or "voice" will be evaluated as the same word.
- use dataverse TinySocial;
-
- for $msg in dataset TweetMessages
- where ftcontains($msg.message-text, "voice", {"mode":"any"})
- return {"id": $msg.id}
-
-The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
-
-The same query can be also expressed in the SQL++.
-
use TinySocial;
- select element {"id":msg.id}
- from TweetMessages as msg
- where TinySocial.ftcontains(msg.`message-text`, "voice", {"mode":"any"})
+ select element {"chirpId": msg.chirpId}
+ from ChirpMessages msg
+ where ftcontains(msg.messageText, "voice", {"mode":"any"});
+
+The DDL and DML of TinySocial can be found in [ADM: Modeling Semistructed Data in AsterixDB](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB).
The `Expression1` is an expression that should be evaluable as a string at runtime as in the above example
-where `$msg.message-text` is a string field. The `Expression2` can be a string, an (un)ordered list
+where `msg.messageText` is a string field. The `Expression2` can be a string, an (un)ordered list
of string value(s), or an expression. In the last case, the given expression should be evaluable
into one of the first two types, i.e., into a string value or an (un)ordered list of string value(s).
The following examples are all valid expressions.
- ... where ftcontains($msg.message-text, "sound")
- ... where ftcontains($msg.message-text, "sound", {"mode":"any"})
- ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
- ... where ftcontains($msg.message-text, {{"speed", "stand", "customization"}}, {"mode":"all"})
- ... where ftcontains($msg.message-text, let $keyword_list := ["voice", "system"] return $keyword_list, {"mode":"all"})
- ... where ftcontains($msg.message-text, $keyword_list, {"mode":"any"})
-
-In the last example above, `$keyword_list` should evaluate to a string or an (un)ordered list of string value(s).
+ ... where ftcontains(msg.messageText, "sound")
+ ... where ftcontains(msg.messageText, "sound", {"mode":"any"})
+ ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
+ ... where ftcontains(msg.messageText, {{"speed", "stand", "customization"}}, {"mode":"all"})
The last `FullTextOption` parameter clarifies the given FTS request. If you omit the `FullTextOption` parameter,
then the default value will be set for each possible option. Currently, we only have one option named `mode`.
And as we extend the FTS feature, more options will be added. Please note that the format of `FullTextOption`
is a record, thus you need to put the option(s) in a record `{}`.
The `mode` option indicates whether the given FTS query is a conjunctive (AND) or disjunctive (OR) search request.
-This option can be either `“any”` or `“all”`. The default value for `mode` is `“all”`. If one specifies `“any”`,
-a disjunctive search will be conducted. For example, the following query will find documents whose `message-text`
+This option can be either `“all”` (AND) or `“any”` (OR). The default value for `mode` is `“all”`. If one specifies `“any”`,
+a disjunctive search will be conducted. For example, the following query will find documents whose `messageText`
field contains “sound” or “system”, so a document will be returned if it contains either “sound”, “system”,
or both of the keywords.
- ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"any"})
+ ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"any"})
The other option parameter,`“all”`, specifies a conjunctive search. The following examples will find the documents whose
-`message-text` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
+`messageText` field contains both “sound” and “system”. If a document contains only “sound” or “system” but
not both, it will not be returned.
- ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
- ... where ftcontains($msg.message-text, ["sound", "system"])
+ ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+ ... where ftcontains(msg.messageText, ["sound", "system"])
Currently AsterixDB doesn’t (yet) support phrase searches, so the following query will not work.
- ... where ftcontains($msg.message-text, "sound system", {"mode":"any"})
+ ... where ftcontains(msg.messageText, "sound system", {"mode":"any"})
As a workaround solution, the following query can be used to achieve a roughly similar goal. The difference is that
-the following queries will find documents where `$msg.message-text` contains both “sound” and “system”, but the order
+the following queries will find documents where `msg.messageText` contains both “sound” and “system”, but the order
and adjacency of “sound” and “system” are not checked, unlike in a phrase search. As a result, the query below would
also return documents with “sound system can be installed.”, “system sound is perfect.”,
or “sound is not clear. You may need to install a new system.”
- ... where ftcontains($msg.message-text, ["sound", "system"], {"mode":"all"})
- ... where ftcontains($msg.message-text, ["sound", "system"])
+ ... where ftcontains(msg.messageText, ["sound", "system"], {"mode":"all"})
+ ... where ftcontains(msg.messageText, ["sound", "system"])
## <a id="FulltextIndex">Creating and utilizing a Full-text index</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
@@ -118,6 +106,9 @@
When there is a full-text index on the field that is being searched, rather than scanning all records,
AsterixDB can utilize that index to expedite the execution of a FTS query. To create a full-text index,
you need to specify the index type as `fulltext` in your DDL statement. For instance, the following DDL
-statement create a full-text index on the TweetMessages.message-text attribute.
+statement create a full-text index on the `GleambookMessages.message` attribute. Note that a full-text index
+cannot be built on a dataset with the variable-length primary key (e.g., string).
- create index messageFTSIdx on TweetMessages(message-text) type fulltext;
+ use TinySocial;
+
+ create index messageFTSIdx on GleambookMessages(message) type fulltext;
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
index 95c752f..43a7cd5 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/manual.md
@@ -283,7 +283,7 @@
### Arithmetic Expressions
AddExpr ::= MultExpr ( ( "+" | "-" ) MultExpr )*
- MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | "^"| "idiv" ) UnaryExpr )*
+ MultExpr ::= UnaryExpr ( ( "*" | "/" | "div" | "%" | "mod" | "^" ) UnaryExpr )*
UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
AQL also supports the usual cast of characters for arithmetic expressions.
diff --git a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
index 0d949db..8118126 100644
--- a/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
+++ b/asterixdb/asterix-doc/src/site/markdown/aql/similarity.md
@@ -45,8 +45,8 @@
AsterixDB supports [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and
[Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets). For
instance, in our
-[TinySocial](primer.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
-example, the `friend-ids` of a Facebook user forms a set
+[TinySocial](../sqlpp/primer-sqlpp.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB)
+example, the `friendIds` of a Gleambook user forms a set
of friends, and we can define a similarity between the sets of
friends of two users. We can also convert a string to a set of grams of a length "n"
(called "n-grams") and define the Jaccard similarity between the two
@@ -55,50 +55,45 @@
`schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
AsterixDB provides
-[tokenization functions](functions.html#Tokenizing_Functions)
+[tokenization functions](../sqlpp/builtins.html#Tokenizing_Functions)
to convert strings to sets, and the
-[similarity functions](functions.html#Similarity_Functions).
+[similarity functions](../sqlpp/builtins.html#Similarity_Functions).
## <a id="SimilaritySelectionQueries">Similarity Selection Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
The following query
-asks for all the Facebook users whose name is similar to
+asks for all the Gleambook users whose name is similar to
`Suzanna Tilson`, i.e., their edit distance is at most 2.
- use dataverse TinySocial;
+ use TinySocial;
- for $user in dataset('FacebookUsers')
- let $ed := edit-distance($user.name, "Suzanna Tilson")
- where $ed <= 2
- return $user
-
+ select u
+ from GleambookUsers u
+ where edit_distance(u.name, "Suzanna Tilson") <= 2;
The following query
-asks for all the Facebook users whose set of friend ids is
+asks for all the Gleambook users whose set of friend ids is
similar to `[1,5,9,10]`, i.e., their Jaccard similarity is at least 0.6.
- use dataverse TinySocial;
+ use TinySocial;
- for $user in dataset('FacebookUsers')
- let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
- where $sim >= 0.6f
- return $user
-
+ select u
+ from GleambookUsers u
+ where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
AsterixDB allows a user to use a similarity operator `~=` to express a
condition by defining the similarity function and threshold
using "set" statements earlier. For instance, the above query can be
equivalently written as:
- use dataverse TinySocial;
+ use TinySocial;
set simfunction "jaccard";
set simthreshold "0.6f";
- for $user in dataset('FacebookUsers')
- where $user.friend-ids ~= [1,5,9,10]
- return $user
-
+ select u
+ from GleambookUsers u
+ where u.friendIds ~= [1,5,9,10];
In this query, we first declare Jaccard as the similarity function
using `simfunction` and then specify the threshold `0.6f` using
@@ -107,27 +102,19 @@
## <a id="SimilarityJoinQueries">Similarity Join Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
AsterixDB supports fuzzy joins between two sets. The following
-[query](primer.html#Query_5_-_Fuzzy_Join)
-finds, for each Facebook user, all Twitter users with names
+[query](../sqlpp/primer-sqlpp.html#Query_5_-_Fuzzy_Join)
+finds, for each Gleambook user, all Chirp users with names
similar to their name based on the edit distance.
- use dataverse TinySocial;
+ use TinySocial;
set simfunction "edit-distance";
set simthreshold "3";
- for $fbu in dataset FacebookUsers
- return {
- "id": $fbu.id,
- "name": $fbu.name,
- "similar-users": for $t in dataset TweetMessages
- let $tu := $t.user
- where $tu.name ~= $fbu.name
- return {
- "twitter-screenname": $tu.screen-name,
- "twitter-name": $tu.name
- }
- };
+ select gbu.id, gbu.name, (select cu.screenName, cu.name
+ from ChirpUsers cu
+ where cu.name ~= gbu.name) as similar_users
+ from GleambookUsers gbu;
## <a id="UsingIndexesToSupportSimilarityQueries">Using Indexes to Support Similarity Queries</a> <font size="4"><a href="#toc">[Back to TOC]</a></font> ##
@@ -146,101 +133,95 @@
[paper](http://www.ics.uci.edu/~chenli/pub/icde2009-memreducer.pdf).
For instance, the following DDL statements create an ngram index on the
-`FacebookUsers.name` attribute using an inverted index of 3-grams.
+`GleambookUsers.name` attribute using an inverted index of 3-grams.
- use dataverse TinySocial;
+ use TinySocial;
- create index fbUserIdx on FacebookUsers(name) type ngram(3);
+ create index gbUserIdx on GleambookUsers(name) type ngram(3);
The number "3" in "ngram(3)" is the length "n" in the grams. This
index can be used to optimize similarity queries on this attribute
using
-[edit-distance](functions.html#edit-distance),
-[edit-distance-check](functions.html#edit-distance-check),
-[similarity-jaccard](functions.html#similarity-jaccard),
-or [similarity-jaccard-check](functions.html#similarity-jaccard-check)
+[edit_distance](../sqlpp/builtins.html#edit_distance),
+[edit_distance_check](../sqlpp/builtins.html#edit_distance_check),
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard),
+or [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check)
queries on this attribute where the
similarity is defined on sets of 3-grams. This index can also be used
-to optimize queries with the "[contains()]((functions.html#contains))" predicate (i.e., substring
+to optimize queries with the "[contains()]((../sqlpp/builtins.html#contains))" predicate (i.e., substring
matching) since it can be also be solved by counting on the inverted
lists of the grams in the query string.
-#### NGram Index usage case - [edit-distance](functions.html#edit-distance) ####
+#### NGram Index usage case - [edit_distance](../sqlpp/builtins.html#edit-distance) ####
- use dataverse TinySocial;
+ use TinySocial;
- for $user in dataset('FacebookUsers')
- let $ed := edit-distance($user.name, "Suzanna Tilson")
- where $ed <= 2
- return $user
+ select u
+ from GleambookUsers u
+ where edit_distance(u.name, "Suzanna Tilson") <= 2;
-#### NGram Index usage case - [edit-distance-check](functions.html#edit-distance-check) ####
+#### NGram Index usage case - [edit_distance_check](../sqlpp/builtins.html#edit_distance_check) ####
- use dataverse TinySocial;
+ use TinySocial;
- for $user in dataset('FacebookUsers')
- let $ed := edit-distance-check($user.name, "Suzanna Tilson", 2)
- where $ed[0]
- return $ed[1]
+ select u
+ from GleambookUsers u
+ where edit_distance_check(u.name, "Suzanna Tilson", 2)[0];
-#### NGram Index usage case - [similarity-jaccard](functions.html#similarity-jaccard) ####
+#### NGram Index usage case - [contains()]((../sqlpp/builtins.html#contains)) ####
- use dataverse TinySocial;
+ use TinySocial;
- for $user in dataset('FacebookUsers')
- let $sim := similarity-jaccard($user.friend-ids, [1,5,9,10])
- where $sim >= 0.6f
- return $user
-
-#### NGram Index usage case - [similarity-jaccard-check](functions.html#similarity-jaccard-check) ####
-
- use dataverse TinySocial;
-
- for $user in dataset('FacebookUsers')
- let $sim := similarity-jaccard-check($user.friend-ids, [1,5,9,10], 0.6f)
- where $sim[0]
- return $user
-
-#### NGram Index usage case - [contains()]((functions.html#contains)) ####
-
- use dataverse TinySocial;
-
- for $i in dataset('FacebookMessages')
- where contains($i.message, "phone")
- return {"mid": $i.message-id, "message": $i.message}
+ select m
+ from GleambookMessages m
+ where contains(m.message, "phone");
### Keyword Index ###
-A "keyword index" is constructed on a set of strings or sets (e.g., OrderedList, UnorderedList). Instead of
+A "keyword index" is constructed on a set of strings or sets (e.g., array, multiset). Instead of
generating grams as in an ngram index, we generate tokens (e.g., words) and for each token, construct an inverted list that includes the ids of the
objects with this token. The following two examples show how to create keyword index on two different types:
#### Keyword Index on String Type ####
- use dataverse TinySocial;
+ use TinySocial;
- drop index FacebookMessages.fbMessageIdx if exists;
- create index fbMessageIdx on FacebookMessages(message) type keyword;
+ drop index GleambookMessages.gbMessageIdx if exists;
+ create index gbMessageIdx on GleambookMessages(message) type keyword;
- for $o in dataset('FacebookMessages')
- let $jacc := similarity-jaccard-check(word-tokens($o.message), word-tokens("love like ccast"), 0.2f)
- where $jacc[0]
- return $o
+ select m
+ from GleambookMessages m
+ where similarity_jaccard_check(word_tokens(m.message), word_tokens("love like ccast"), 0.2f)[0];
-#### Keyword Index on UnorderedList Type ####
+#### Keyword Index on Multiset Type ####
- use dataverse TinySocial;
+ use TinySocial;
- create index fbUserIdx_fids on FacebookUsers(friend-ids) type keyword;
+ create index gbUserIdxFIds on GleambookUsers(friendIds) type keyword;
- for $c in dataset('FacebookUsers')
- let $jacc := similarity-jaccard-check($c.friend-ids, {{3,10}}, 0.5f)
- where $jacc[0]
- return $c
+ select u
+ from GleambookUsers u
+ where similarity_jaccard_check(u.friendIds, {{3,10}}, 0.5f)[0];
As shown above, keyword index can be used to optimize queries with token-based similarity predicates, including
-[similarity-jaccard](functions.html#similarity-jaccard) and
-[similarity-jaccard-check](functions.html#similarity-jaccard-check).
+[similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) and
+[similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check).
+
+#### Keyword Index usage case - [similarity_jaccard](../sqlpp/builtins.html#similarity_jaccard) ####
+
+ use TinySocial;
+
+ select u
+ from GleambookUsers u
+ where similarity_jaccard(u.friendIds, [1,5,9,10]) >= 0.6f;
+
+#### Keyword Index usage case - [similarity_jaccard_check](../sqlpp/builtins.html#similarity_jaccard_check) ####
+
+ use TinySocial;
+
+ select u
+ from GleambookUsers u
+ where similarity_jaccard_check(u.friendIds, [1,5,9,10], 0.6f)[0];
diff --git a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
index d2d4488..f5635b8 100644
--- a/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
+++ b/asterixdb/asterix-doc/src/site/markdown/feeds/tutorial.md
@@ -23,7 +23,7 @@
* [Introduction](#Introduction)
* [Feed Adapters](#FeedAdapters)
-<!-- * [Feed Policies](#FeedPolicies) -->
+* [Feed Policies](#FeedPolicies)
## <a name="Introduction">Introduction</a> ##
@@ -61,11 +61,13 @@
####Ingesting Twitter Stream
We shall use the built-in push-based Twitter adapter.
As a pre-requisite, we must define a Tweet using the AsterixDB Data Model (ADM)
-and the AsterixDB Query Language (AQL). Given below are the type definitions in AQL
+and the query language SQL++. Given below are the type definitions in SQL++
that create a Tweet datatype which is representative of a real tweet as obtained from Twitter.
+ drop dataverse feeds if exists;
+
create dataverse feeds;
- use dataverse feeds;
+ use feeds;
create type TwitterUser as closed {
screen_name: string,
@@ -77,13 +79,12 @@
create type Tweet as open {
id: int64,
user: TwitterUser
- }
+ };
- create dataset Tweets (Tweet)
- primary key id;
+ create dataset Tweets (Tweet) primary key id;
We also create a dataset that we shall use to persist the tweets in AsterixDB.
-Next we make use of the `create feed` AQL statement to define our example data feed.
+Next we make use of the `create feed` SQL++ statement to define our example data feed.
#####Using the "push_twitter" feed adapter#####
The "push_twitter" adapter requires setting up an application account with Twitter. To retrieve
@@ -91,6 +92,7 @@
a name and a brief description for the application. Each application has associated OAuth
authentication credentials that include OAuth keys and tokens. Accessing the
Twitter API requires providing the following.
+
1. Consumer Key (API Key)
2. Consumer Secret (API Secret)
3. Access Token
@@ -101,18 +103,20 @@
using the "push_twitter" adapter. For further information on obtaining OAuth keys and tokens and
registering an application with Twitter, please visit http://apps.twitter.com
-Given below is an example AQL statement that creates a feed called "TwitterFeed" by using the
+Given below is an example SQL++ statement that creates a feed called "TwitterFeed" by using the
"push_twitter" adapter.
- use dataverse feeds;
+ use feeds;
- create feed TwitterFeed if not exists using "push_twitter"
- (("type-name"="Tweet"),
- ("format"="twitter-status"),
- ("consumer.key"="************"),
- ("consumer.secret"="**************"),
- ("access.token"="**********"),
- ("access.token.secret"="*************"));
+ create feed TwitterFeed with {
+ "adapter-name": "push_twitter",
+ "type-name": "Tweet",
+ "format": "twitter-status",
+ "consumer.key": "************",
+ "consumer.secret": "************",
+ "access.token": "**********",
+ "access.token.secret": "*************"
+ };
It is required that the above authentication parameters are provided valid.
Note that the `create feed` statement does not initiate the flow of data from Twitter into
@@ -122,23 +126,25 @@
The Twitter adapter also supports several Twitter streaming APIs as follow:
-1. Track filter ("keywords"="AsterixDB, Apache")
-2. Locations filter ("locations"="-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358")
-3. Language filter ("language"="en")
-4. Filter level ("filter-level"="low")
+1. Track filter `"keywords": "AsterixDB, Apache"`
+2. Locations filter `"locations": "-29.7, 79.2, 36.7, 72.0; -124.848974,-66.885444, 24.396308, 49.384358"`
+3. Language filter `"language": "en"`
+4. Filter level `"filter-level": "low"`
An example of Twitter adapter tracking tweets with keyword "news" can be described using following ddl:
- use dataverse feeds;
+ use feeds;
- create feed TwitterFeed if not exists using "push_twitter"
- (("type-name"="Tweet"),
- ("format"="twitter-status"),
- ("consumer.key"="************"),
- ("consumer.secret"="**************"),
- ("access.token"="**********"),
- ("access.token.secret"="*************"),
- ("keywords"="news"));
+ create feed TwitterFeed with {
+ "adapter-name": "push_twitter",
+ "type-name": "Tweet",
+ "format": "twitter-status",
+ "consumer.key": "************",
+ "consumer.secret": "************",
+ "access.token": "**********",
+ "access.token.secret": "*************",
+ "keywords": "news"
+ };
For more details about these APIs, please visit https://dev.twitter.com/streaming/overview/request-parameters
@@ -154,7 +160,7 @@
contents of the dataset represent the union of the connected feeds.
Also one feed can be simultaneously connected to multiple target datasets.
- use dataverse feeds;
+ use feeds;
connect feed TwitterFeed to dataset Tweets;
@@ -170,108 +176,53 @@
Let the feed run for a minute, then run the following query to see the
latest tweets that are stored into the data set.
- use dataverse feeds;
+ use feeds;
- for $i in dataset Tweets limit 10 return $i;
+ select * from Tweets limit 10;
The dataflow of data from a feed can be terminated explicitly by `stop feed` statement.
- use dataverse feeds;
+ use feeds;
stop feed TwitterFeed;
The `disconnnect statement` can be used to disconnect the feed from certain dataset.
- use dataverse feeds;
+ use feeds;
disconnect feed TwitterFeed from dataset Tweets;
###Ingesting with Other Adapters
AsterixDB has several builtin feed adapters for data ingestion. User can also
implement their own adapters and plug them into AsterixDB.
-Here we introduce `rss_feed`, `socket_adapter` and `localfs`
+Here we introduce `socket_adapter` and `localfs`
feed adapter that cover most of the common application scenarios.
-#####Using the "rss_feed" feed adapter#####
-`rss_feed` adapter allows retrieving data given a collection of RSS end point URLs.
-As observed in the case of ingesting tweets, it is required to model an RSS data item using AQL.
-
- use dataverse feeds;
-
- create type Rss if not exists as open {
- id: string,
- title: string,
- description: string,
- link: string
- };
-
- create dataset RssDataset (Rss)
- primary key id;
-
-Next, we define an RSS feed using our built-in adapter "rss_feed".
-
- use dataverse feeds;
-
- create feed my_feed using
- rss_feed (
- ("type-name"="Rss"),
- ("format"="rss"),
- ("url"="http://rss.cnn.com/rss/edition.rss")
- );
-
-In the above definition, the configuration parameter "url" can be a comma-separated list that reflects a
-collection of RSS URLs, where each URL corresponds to an RSS endpoint or an RSS feed.
-The "rss_feed" retrieves data from each of the specified RSS URLs (comma separated values) in parallel.
-
-The following statements connect the feed into the `RssDataset`:
-
- use dataverse feeds;
-
- connect feed my_feed to dataset RssDataset;
-
-The following statements activate the feed and start the dataflow:
-
- use dataverse feeds;
-
- start feed my_feed;
-
-The following statements show the latest data from the data set, stop the feed, and
-disconnect the feed from the data set.
-
- use dataverse feeds;
-
- for $i in dataset RssDataset limit 10 return $i;
-
- stop feed my_feed
-
- disconnect feed my_feed from dataset RssDataset;
-
-
#####Using the "socket_adapter" feed adapter#####
`socket_adapter` feed opens a web socket on the given node which allows user to push data into
AsterixDB directly. Here is an example:
drop dataverse feeds if exists;
create dataverse feeds;
- use dataverse feeds;
+ use feeds;
create type TestDataType as open {
screenName: string
- }
+ };
create dataset TestDataset(TestDataType) primary key screenName;
- create feed TestSocketFeed using socket_adapter
- (
- ("sockets"="127.0.0.1:10001"),
- ("address-type"="IP"),
- ("type-name"="TestDataType"),
- ("format"="adm")
- );
+ create feed TestSocketFeed with {
+ "adapter-name": "socket_adapter",
+ "sockets": "127.0.0.1:10001",
+ "address-type": "IP",
+ "type-name": "TestDataType",
+ "format": "adm"
+ };
connect feed TestSocketFeed to dataset TestDataset;
- use dataverse feeds;
+ use feeds;
start feed TestSocketFeed;
The above statements create a socket feed which is listening to "10001" port of the host machine. This feed accepts data
@@ -297,25 +248,27 @@
`localfs` adapter enables data ingestion from local file system. It allows user to feed data records on local disk
into a dataset. A DDL example for creating a `localfs` feed is given as follow:
- use dataverse feeds;
+ use feeds;
- create type TweetType as closed {
- id: string,
- username : string,
- location : string,
- text : string,
- timestamp : string
- }
+ create type TestDataType as open {
+ screenName: string
+ };
- create dataset Tweets(TweetType)
- primary key id;
+ create dataset TestDataset(TestDataType) primary key screenName;
- create feed TweetFeed
- using localfs
- (("type-name"="TweetType"),("path"="HOSTNAME://LOCAL_FILE_PATH"),("format"="adm"))
+ create feed TestFileFeed with {
+ "adapter-name": "localfs",
+ "type-name": "TestDataType",
+ "path": "HOSTNAME://LOCAL_FILE_PATH",
+ "format": "adm"
+ };
+
+ connect feed TestFileFeed to dataset TestDataset;
+
+ start feed TestFileFeed;
Similar to previous examples, we need to define the datatype and dataset this feed uses.
-The "path" parameter refers to the local datafile that we want to ingest data from.
+The "path" parameter refers to the local data file that we want to ingest data from.
`HOSTNAME` can either be the IP address or node name of the machine which holds the file.
`LOCAL_FILE_PATH` indicates the absolute path to the file on that machine. Similarly to `socket_adapter`,
this feed takes `adm` formatted data records.
@@ -334,7 +287,7 @@
Use that same datatype in feed definition will cause a type discrepancy since there is no such field in the datasource.
Thus, we will need to define two separate datatypes for feed and dataset:
- use dataverse feeds;
+ use feeds;
create type DBLPFeedType as closed {
dblpid: string,
@@ -352,13 +305,13 @@
}
create dataset DBLPDataset(DBLPDataSetType) primary key id autogenerated;
- create feed DBLPFeed using socket_adapter
- (
- ("sockets"="127.0.0.1:10001"),
- ("address-type"="IP"),
- ("type-name"="DBLPFeedType"),
- ("format"="adm")
- );
+ create feed DBLPFeed with {
+ "adapter-name": "socket_adapter",
+ "sockets": "127.0.0.1:10001",
+ "address-type": "IP",
+ "type-name": "DBLPFeedType",
+ "format": "adm"
+ };
connect feed DBLPFeed to dataset DBLPDataset;
@@ -403,7 +356,6 @@
ingestion policy is specified as part of the `connect feed` statement
or else the "Basic" policy will be chosen as the default.
- use dataverse feeds;
+ use feeds;
- connect feed TwitterFeed to dataset Tweets
- using policy Basic;
\ No newline at end of file
+ connect feed TwitterFeed to dataset Tweets using policy Basic;
\ No newline at end of file
diff --git a/asterixdb/asterix-doc/src/site/markdown/ncservice.md b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
index 605ee30..2b309ce 100644
--- a/asterixdb/asterix-doc/src/site/markdown/ncservice.md
+++ b/asterixdb/asterix-doc/src/site/markdown/ncservice.md
@@ -366,7 +366,7 @@
| common | txn.lock.timeout.sweepthreshold | Interval (in milliseconds) for checking lock timeout | 10000 |
| common | txn.lock.timeout.waitthreshold | Time out (in milliseconds) of waiting for a lock | 60000 |
| common | txn.log.buffer.numpages | The number of pages in the transaction log tail | 8 |
-| common | txn.log.buffer.pagesize | The page size (in bytes) for transaction log buffer | 131072 (128 kB) |
+| common | txn.log.buffer.pagesize | The page size (in bytes) for transaction log buffer | 4194304 (4MB) |
| common | txn.log.checkpoint.history | The number of checkpoints to keep in the transaction log | 0 |
| common | txn.log.checkpoint.lsnthreshold | The checkpoint threshold (in terms of LSNs (log sequence numbers) that have been written to the transaction log, i.e., the length of the transaction log) for transaction logs | 67108864 (64 MB) |
| common | txn.log.checkpoint.pollfrequency | The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be written | 120 |
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 78549b3..cbaa889 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -397,5 +397,10 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.esri.geometry</groupId>
+ <artifactId>esri-geometry-api</artifactId>
+ <version>2.0.0</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
deleted file mode 100644
index 23c5bdd..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordWithMetaDataAndPKParser.java
+++ /dev/null
@@ -1,32 +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.external.api;
-
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-
-public interface IRecordWithMetaDataAndPKParser<T> extends IRecordDataParser<RecordWithMetadataAndPK<T>> {
-
- public void parseMeta(RecordWithMetadataAndPK<? extends T> record, DataOutput out) throws IOException;
-
- public void appendKeys(RecordWithMetadataAndPK<T> record, ArrayTupleBuilder tb) throws IOException;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
deleted file mode 100755
index 9f14ec0..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IResultCollector.java
+++ /dev/null
@@ -1,45 +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.external.api;
-
-import java.io.DataOutput;
-
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.om.base.AOrderedList;
-import org.apache.asterix.om.base.ARecord;
-import org.apache.asterix.om.base.IAObject;
-
-public interface IResultCollector {
-
- public void writeIntResult(int result) throws AsterixException;
-
- public void writeFloatResult(float result) throws AsterixException;
-
- public void writeDoubleResult(double result) throws AsterixException;
-
- public void writeStringResult(String result) throws AsterixException;
-
- public void writeRecordResult(ARecord result) throws AsterixException;
-
- public void writeListResult(AOrderedList list) throws AsterixException;
-
- public IAObject getComplexTypeResultHolder();
-
- public DataOutput getDataOutput();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
index 79dc396..df4c093 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexCreateOperatorDescriptor.java
@@ -18,8 +18,11 @@
*/
package org.apache.asterix.external.operators;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -36,7 +39,8 @@
import org.apache.hyracks.storage.am.common.api.IIndexBuilderFactory;
import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import org.apache.hyracks.storage.common.IIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
/**
@@ -74,10 +78,13 @@
// Open the index
indexHelper.open();
try {
- IIndex index = indexHelper.getIndexInstance();
+ ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+ LSMComponentId.DEFAULT_COMPONENT_ID);
// Create bulk loader
IIndexBulkLoader bulkLoader =
- index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false);
+ index.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), false, parameters);
// Load files
for (ExternalFile file : files) {
bulkLoader.add(filesTupleTranslator.getTupleFromFile(file));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
index 4bc2867..3bada4a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
@@ -18,7 +18,9 @@
*/
package org.apache.asterix.external.operators;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.indexing.ExternalFile;
@@ -69,9 +71,10 @@
indexHelper.open();
IIndex index = indexHelper.getIndexInstance();
LSMTwoPCBTreeBulkLoader bulkLoader = null;
+ Map<String, Object> parameters = new HashMap<>();
try {
bulkLoader = (LSMTwoPCBTreeBulkLoader) ((ExternalBTree) index)
- .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size());
+ .createTransactionBulkLoader(BTree.DEFAULT_FILL_FACTOR, false, files.size(), parameters);
// Load files
// The files must be ordered according to their numbers
for (ExternalFile file : files) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
index 573de5d..74bc0dc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkLoadOperatorNodePushable.java
@@ -18,12 +18,18 @@
*/
package org.apache.asterix.external.operators;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
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.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexBulkLoadOperatorNodePushable;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
public class ExternalIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
@@ -43,4 +49,12 @@
super.open();
((ITwoPCIndex) index).setCurrentVersion(version);
}
+
+ @Override
+ protected void initializeBulkLoader() throws HyracksDataException {
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
+ bulkLoader = ((ILSMIndex) index).createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ parameters);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
index aaca3f1..57e2917 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalIndexBulkModifyOperatorNodePushable.java
@@ -19,6 +19,8 @@
package org.apache.asterix.external.operators;
import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.asterix.external.indexing.FilesIndexDescription;
import org.apache.asterix.om.base.AMutableInt32;
@@ -60,8 +62,9 @@
try {
writer.open();
// Transactional BulkLoader
- bulkLoader =
- ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length);
+ Map<String, Object> parameters = new HashMap<>();
+ bulkLoader = ((ITwoPCIndex) index).createTransactionBulkLoader(fillFactor, verifyInput, deletedFiles.length,
+ parameters);
// Delete files
for (int i = 0; i < deletedFiles.length; i++) {
fileNumber.setValue(deletedFiles[i]);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index c0ccf11..f1eba4c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
-import java.io.Serializable;
import java.util.BitSet;
import java.util.List;
@@ -52,6 +51,7 @@
import org.apache.asterix.om.util.container.IObjectPool;
import org.apache.asterix.om.util.container.ListObjectPool;
import org.apache.asterix.runtime.operators.file.adm.AdmLexer;
+import org.apache.asterix.runtime.operators.file.adm.AdmLexer.TokenImage;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IMutableValueStorage;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -72,64 +72,11 @@
private final IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
new ListObjectPool<IMutableValueStorage, ATypeTag>(new AbvsBuilderFactory());
+ private final TokenImage tmpTokenImage = new TokenImage();
+
private final String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
private final String mismatchErrorMessage2 = " got a value of type ";
- static class ParseException extends HyracksDataException {
- private static final long serialVersionUID = 1L;
- private String filename;
- private int line = -1;
- private int column = -1;
-
- public ParseException(String message) {
- super(message);
- }
-
- public ParseException(int errorCode, Serializable... param) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
- }
-
- public ParseException(int errorCode, Throwable e, Serializable... param) {
- super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
- addSuppressed(e);
- }
-
- public ParseException(Throwable cause) {
- super(cause);
- }
-
- public ParseException(String message, Throwable cause) {
- super(message, cause);
- }
-
- public ParseException(Throwable cause, String filename, int line, int column) {
- super(cause);
- setLocation(filename, line, column);
- }
-
- public void setLocation(String filename, int line, int column) {
- this.filename = filename;
- this.line = line;
- this.column = column;
- }
-
- @Override
- public String getMessage() {
- StringBuilder msg = new StringBuilder("Parse error");
- if (filename != null) {
- msg.append(" in file " + filename);
- }
- if (line >= 0) {
- if (column >= 0) {
- msg.append(" at (" + line + ", " + column + ")");
- } else {
- msg.append(" in line " + line);
- }
- }
- return msg.append(": " + super.getMessage()).toString();
- }
- }
-
public ADMDataParser(ARecordType recordType, boolean isStream) {
this(null, recordType, isStream);
}
@@ -254,16 +201,26 @@
break;
case AdmLexer.TOKEN_STRING_LITERAL:
if (checkType(ATypeTag.STRING, objectType)) {
- String tokenImage =
- admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
- aString.setValue(admLexer.containsEscapes() ? replaceEscapes(tokenImage) : tokenImage);
- stringSerde.serialize(aString, out);
+ admLexer.getLastTokenImage(tmpTokenImage);
+ if (admLexer.containsEscapes()) {
+ replaceEscapes(tmpTokenImage);
+ }
+ int begin = tmpTokenImage.getBegin() + 1;
+ int len = tmpTokenImage.getLength() - 2;
+ parseString(tmpTokenImage.getBuffer(), begin, len, out);
} else if (checkType(ATypeTag.UUID, objectType)) {
// Dealing with UUID type that is represented by a string
+ admLexer.getLastTokenImage(tmpTokenImage);
+ aUUID.parseUUIDString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+ tmpTokenImage.getLength() - 2);
+ uuidSerde.serialize(aUUID, out);
+ } else if (checkType(ATypeTag.GEOMETRY, objectType)) {
+ // Parse the string as a WKT-encoded geometry
String tokenImage =
admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
- aUUID.parseUUIDString(tokenImage);
- uuidSerde.serialize(aUUID, out);
+ aGeomtry.parseWKT(tokenImage);
+ out.writeByte(ATypeTag.GEOMETRY.serialize());
+ geomSerde.serialize(aGeomtry, out);
} else {
throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_TYPE_MISMATCH, objectType.getTypeName());
}
@@ -364,13 +321,14 @@
}
- private String replaceEscapes(String tokenImage) throws ParseException {
- char[] chars = tokenImage.toCharArray();
- int len = chars.length;
- int readpos = 0;
- int writepos = 0;
- int movemarker = 0;
- while (readpos < len) {
+ // TODO: This function should be optimized. Currently it has complexity of O(N*N)!
+ private void replaceEscapes(TokenImage tokenImage) throws ParseException {
+ char[] chars = tokenImage.getBuffer();
+ int end = tokenImage.getBegin() + tokenImage.getLength();
+ int readpos = tokenImage.getBegin();
+ int writepos = tokenImage.getBegin();
+ int movemarker = tokenImage.getBegin();
+ while (readpos < end) {
if (chars[readpos] == '\\') {
moveChars(chars, movemarker, readpos, readpos - writepos);
switch (chars[readpos + 1]) {
@@ -407,8 +365,8 @@
++writepos;
++readpos;
}
- moveChars(chars, movemarker, len, readpos - writepos);
- return new String(chars, 0, len - (readpos - writepos));
+ moveChars(chars, movemarker, end, readpos - writepos);
+ tokenImage.reset(chars, tokenImage.getBegin(), tokenImage.getLength() - (readpos - writepos));
}
private static void moveChars(char[] chars, int start, int end, int offset) {
@@ -508,16 +466,16 @@
expectingRecordField = false;
if (recType != null) {
- String fldName =
- admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
+ admLexer.getLastTokenImage(tmpTokenImage);
+ String fldName = new String(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+ tmpTokenImage.getLength() - 2);
fieldId = recBuilder.getFieldId(fldName);
if ((fieldId < 0) && !recType.isOpen()) {
throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_EXTRA_FIELD_IN_CLOSED_RECORD,
fldName);
} else if ((fieldId < 0) && recType.isOpen()) {
- aStringFieldName.setValue(admLexer.getLastTokenImage().substring(1,
- admLexer.getLastTokenImage().length() - 1));
- stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+ parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+ tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
openRecordField = true;
fieldType = null;
} else {
@@ -527,9 +485,9 @@
openRecordField = false;
}
} else {
- aStringFieldName.setValue(
- admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1));
- stringSerde.serialize(aStringFieldName, fieldNameBuffer.getDataOutput());
+ admLexer.getLastTokenImage(tmpTokenImage);
+ parseString(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin() + 1,
+ tmpTokenImage.getLength() - 2, fieldNameBuffer.getDataOutput());
openRecordField = true;
fieldType = null;
}
@@ -807,7 +765,13 @@
private void parseToNumericTarget(ATypeTag typeTag, IAType objectType, DataOutput out) throws IOException {
ATypeTag targetTypeTag = getTargetTypeTag(typeTag, objectType);
- if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), targetTypeTag, out)) {
+ boolean parsed = false;
+ if (targetTypeTag != null) {
+ admLexer.getLastTokenImage(tmpTokenImage);
+ parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(),
+ targetTypeTag, out);
+ }
+ if (!parsed) {
throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
}
}
@@ -819,8 +783,13 @@
castBuffer.reset();
dataOutput = castBuffer.getDataOutput();
}
-
- if ((targetTypeTag == null) || !parseValue(admLexer.getLastTokenImage(), typeTag, dataOutput)) {
+ boolean parsed = false;
+ if (targetTypeTag != null) {
+ admLexer.getLastTokenImage(tmpTokenImage);
+ parsed = parseValue(tmpTokenImage.getBuffer(), tmpTokenImage.getBegin(), tmpTokenImage.getLength(), typeTag,
+ dataOutput);
+ }
+ if (!parsed) {
throw new ParseException(mismatchErrorMessage + objectType.getTypeName() + mismatchErrorMessage2 + typeTag);
}
@@ -862,9 +831,11 @@
if (token == AdmLexer.TOKEN_CONSTRUCTOR_OPEN) {
token = admLexer.next();
if (token == AdmLexer.TOKEN_STRING_LITERAL) {
- String unquoted =
- admLexer.getLastTokenImage().substring(1, admLexer.getLastTokenImage().length() - 1);
- if (!parseValue(unquoted, typeTag, dataOutput)) {
+ admLexer.getLastTokenImage(tmpTokenImage);
+ int begin = tmpTokenImage.getBegin() + 1;
+ int len = tmpTokenImage.getLength() - 2;
+ // unquoted value
+ if (!parseValue(tmpTokenImage.getBuffer(), begin, len, typeTag, dataOutput)) {
throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_CONSTRUCTOR_MISSING_DESERIALIZER,
AdmLexer.tokenKindToString(token));
}
@@ -890,85 +861,86 @@
objectType.getTypeName() + " got " + typeTag);
}
- private boolean parseValue(String unquoted, ATypeTag typeTag, DataOutput out) throws HyracksDataException {
+ private boolean parseValue(char[] buffer, int begin, int len, ATypeTag typeTag, DataOutput out)
+ throws HyracksDataException {
switch (typeTag) {
case BOOLEAN:
- parseBoolean(unquoted, out);
+ parseBoolean(buffer, begin, len, out);
return true;
case TINYINT:
- parseInt8(unquoted, out);
+ parseInt8(buffer, begin, len, out);
return true;
case SMALLINT:
- parseInt16(unquoted, out);
+ parseInt16(buffer, begin, len, out);
return true;
case INTEGER:
- parseInt32(unquoted, out);
+ parseInt32(buffer, begin, len, out);
return true;
case BIGINT:
- parseInt64(unquoted, out);
+ parseInt64(buffer, begin, len, out);
return true;
case FLOAT:
- if ("INF".equals(unquoted)) {
+ if (matches("INF", buffer, begin, len)) {
aFloat.setValue(Float.POSITIVE_INFINITY);
- } else if ("-INF".equals(unquoted)) {
+ } else if (matches("-INF", buffer, begin, len)) {
aFloat.setValue(Float.NEGATIVE_INFINITY);
} else {
- aFloat.setValue(Float.parseFloat(unquoted));
+ aFloat.setValue(parseFloat(buffer, begin, len));
}
floatSerde.serialize(aFloat, out);
return true;
case DOUBLE:
- if ("INF".equals(unquoted)) {
+ if (matches("INF", buffer, begin, len)) {
aDouble.setValue(Double.POSITIVE_INFINITY);
- } else if ("-INF".equals(unquoted)) {
+ } else if (matches("-INF", buffer, begin, len)) {
aDouble.setValue(Double.NEGATIVE_INFINITY);
} else {
- aDouble.setValue(Double.parseDouble(unquoted));
+ aDouble.setValue(parseDouble(buffer, begin, len));
}
doubleSerde.serialize(aDouble, out);
return true;
case STRING:
- aString.setValue(unquoted);
- stringSerde.serialize(aString, out);
+ parseString(buffer, begin, len, out);
return true;
case TIME:
- parseTime(unquoted, out);
+ parseTime(buffer, begin, len, out);
return true;
case DATE:
- parseDate(unquoted, out);
+ parseDate(buffer, begin, len, out);
return true;
case DATETIME:
- parseDateTime(unquoted, out);
+ parseDateTime(buffer, begin, len, out);
return true;
case DURATION:
- parseDuration(unquoted, out);
+ parseDuration(buffer, begin, len, out);
return true;
case DAYTIMEDURATION:
- parseDateTimeDuration(unquoted, out);
+ parseDateTimeDuration(buffer, begin, len, out);
return true;
case YEARMONTHDURATION:
- parseYearMonthDuration(unquoted, out);
+ parseYearMonthDuration(buffer, begin, len, out);
return true;
case POINT:
- parsePoint(unquoted, out);
+ parsePoint(buffer, begin, len, out);
return true;
case POINT3D:
- parse3DPoint(unquoted, out);
+ parse3DPoint(buffer, begin, len, out);
return true;
case CIRCLE:
- parseCircle(unquoted, out);
+ parseCircle(buffer, begin, len, out);
return true;
case RECTANGLE:
- parseRectangle(unquoted, out);
+ parseRectangle(buffer, begin, len, out);
return true;
case LINE:
- parseLine(unquoted, out);
+ parseLine(buffer, begin, len, out);
return true;
case POLYGON:
- APolygonSerializerDeserializer.parse(unquoted, out);
+ //TODO: optimize
+ APolygonSerializerDeserializer.parse(new String(buffer, begin, len), out);
return true;
case UUID:
- aUUID.parseUUIDString(unquoted);
+ aUUID.parseUUIDString(buffer, begin, len);
uuidSerde.serialize(aUUID, out);
return true;
default:
@@ -976,39 +948,53 @@
}
}
- private void parseBoolean(String bool, DataOutput out) throws HyracksDataException {
- if (bool.equals("true")) {
+ private boolean matches(String value, char[] buffer, int begin, int len) {
+ if (len != value.length()) {
+ return false;
+ }
+ for (int i = 0; i < len; i++) {
+ if (value.charAt(i) != buffer[i + begin]) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private void parseBoolean(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+ if (matches("true", buffer, begin, len)) {
booleanSerde.serialize(ABoolean.TRUE, out);
- } else if (bool.equals("false")) {
+ } else if (matches("false", buffer, begin, len)) {
booleanSerde.serialize(ABoolean.FALSE, out);
} else {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "boolean");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ "boolean");
}
}
- private void parseInt8(String int8, DataOutput out) throws HyracksDataException {
+ private void parseInt8(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
boolean positive = true;
byte value = 0;
- int offset = 0;
+ int offset = begin;
- if (int8.charAt(offset) == '+') {
+ if (buffer[offset] == '+') {
offset++;
- } else if (int8.charAt(offset) == '-') {
+ } else if (buffer[offset] == '-') {
offset++;
positive = false;
}
- for (; offset < int8.length(); offset++) {
- if ((int8.charAt(offset) >= '0') && (int8.charAt(offset) <= '9')) {
- value = (byte) (((value * 10) + int8.charAt(offset)) - '0');
- } else if ((int8.charAt(offset) == 'i') && (int8.charAt(offset + 1) == '8')
- && ((offset + 2) == int8.length())) {
+ for (; offset < begin + len; offset++) {
+ if ((buffer[offset] >= '0') && (buffer[offset] <= '9')) {
+ value = (byte) (((value * 10) + buffer[offset]) - '0');
+ } else if (buffer[offset] == 'i' && buffer[offset + 1] == '8' && offset + 2 == begin + len) {
break;
} else {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+ new String(buffer, begin, len), "int8");
}
}
if (value < 0) {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int8");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ "int8");
}
if ((value > 0) && !positive) {
value *= -1;
@@ -1017,29 +1003,31 @@
int8Serde.serialize(aInt8, out);
}
- private void parseInt16(String int16, DataOutput out) throws HyracksDataException {
+ private void parseInt16(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
boolean positive = true;
short value = 0;
- int offset = 0;
+ int offset = begin;
- if (int16.charAt(offset) == '+') {
+ if (buffer[offset] == '+') {
offset++;
- } else if (int16.charAt(offset) == '-') {
+ } else if (buffer[offset] == '-') {
offset++;
positive = false;
}
- for (; offset < int16.length(); offset++) {
- if ((int16.charAt(offset) >= '0') && (int16.charAt(offset) <= '9')) {
- value = (short) (((value * 10) + int16.charAt(offset)) - '0');
- } else if ((int16.charAt(offset) == 'i') && (int16.charAt(offset + 1) == '1')
- && (int16.charAt(offset + 2) == '6') && ((offset + 3) == int16.length())) {
+ for (; offset < begin + len; offset++) {
+ if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+ value = (short) ((value * 10) + buffer[offset] - '0');
+ } else if (buffer[offset] == 'i' && buffer[offset + 1] == '1' && buffer[offset + 2] == '6'
+ && offset + 3 == begin + len) {
break;
} else {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+ new String(buffer, begin, len), "int16");
}
}
if (value < 0) {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int16");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ "int16");
}
if ((value > 0) && !positive) {
value *= -1;
@@ -1048,29 +1036,31 @@
int16Serde.serialize(aInt16, out);
}
- private void parseInt32(String int32, DataOutput out) throws HyracksDataException {
+ private void parseInt32(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
boolean positive = true;
int value = 0;
- int offset = 0;
+ int offset = begin;
- if (int32.charAt(offset) == '+') {
+ if (buffer[offset] == '+') {
offset++;
- } else if (int32.charAt(offset) == '-') {
+ } else if (buffer[offset] == '-') {
offset++;
positive = false;
}
- for (; offset < int32.length(); offset++) {
- if ((int32.charAt(offset) >= '0') && (int32.charAt(offset) <= '9')) {
- value = (((value * 10) + int32.charAt(offset)) - '0');
- } else if ((int32.charAt(offset) == 'i') && (int32.charAt(offset + 1) == '3')
- && (int32.charAt(offset + 2) == '2') && ((offset + 3) == int32.length())) {
+ for (; offset < begin + len; offset++) {
+ if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+ value = (value * 10) + buffer[offset] - '0';
+ } else if (buffer[offset] == 'i' && buffer[offset + 1] == '3' && buffer[offset + 2] == '2'
+ && offset + 3 == begin + len) {
break;
} else {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+ new String(buffer, begin, len), "int32");
}
}
if (value < 0) {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int32");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ "int32");
}
if ((value > 0) && !positive) {
value *= -1;
@@ -1080,29 +1070,31 @@
int32Serde.serialize(aInt32, out);
}
- private void parseInt64(String int64, DataOutput out) throws HyracksDataException {
+ private void parseInt64(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
boolean positive = true;
long value = 0;
- int offset = 0;
+ int offset = begin;
- if (int64.charAt(offset) == '+') {
+ if (buffer[offset] == '+') {
offset++;
- } else if (int64.charAt(offset) == '-') {
+ } else if (buffer[offset] == '-') {
offset++;
positive = false;
}
- for (; offset < int64.length(); offset++) {
- if ((int64.charAt(offset) >= '0') && (int64.charAt(offset) <= '9')) {
- value = (((value * 10) + int64.charAt(offset)) - '0');
- } else if ((int64.charAt(offset) == 'i') && (int64.charAt(offset + 1) == '6')
- && (int64.charAt(offset + 2) == '4') && ((offset + 3) == int64.length())) {
+ for (; offset < begin + len; offset++) {
+ if (buffer[offset] >= '0' && buffer[offset] <= '9') {
+ value = (value * 10) + buffer[offset] - '0';
+ } else if (buffer[offset] == 'i' && buffer[offset + 1] == '6' && buffer[offset + 2] == '4'
+ && offset + 3 == begin + len) {
break;
} else {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE,
+ new String(buffer, begin, len), "int64");
}
}
if (value < 0) {
- throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, "int64");
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, new String(buffer, begin, len),
+ "int64");
}
if ((value > 0) && !positive) {
value *= -1;
@@ -1127,4 +1119,4 @@
admLexer.reInit(new InputStreamReader(in));
return true;
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
index 8351931..859ac22 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/AbstractDataParser.java
@@ -19,7 +19,10 @@
package org.apache.asterix.external.parser;
import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
import org.apache.asterix.external.api.IDataParser;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.ABinary;
@@ -31,6 +34,7 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.base.AInt16;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt64;
@@ -45,6 +49,7 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableDuration;
import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableGeometry;
import org.apache.asterix.om.base.AMutableInt16;
import org.apache.asterix.om.base.AMutableInt32;
import org.apache.asterix.om.base.AMutableInt64;
@@ -71,11 +76,14 @@
import org.apache.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
import org.apache.asterix.om.base.temporal.ATimeParserFactory;
import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
+import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.util.bytes.Base64Parser;
import org.apache.hyracks.util.bytes.HexParser;
+import org.apache.hyracks.util.string.UTF8StringReader;
+import org.apache.hyracks.util.string.UTF8StringWriter;
/**
* Base class for data parsers. Includes the common set of definitions for
@@ -93,6 +101,7 @@
protected AMutableBinary aBinary = new AMutableBinary(null, 0, 0);
protected AMutableString aStringFieldName = new AMutableString("");
protected AMutableUUID aUUID = new AMutableUUID();
+ protected AMutableGeometry aGeomtry = new AMutableGeometry(null);
// For temporal and spatial data types
protected AMutableTime aTime = new AMutableTime(0);
protected AMutableDateTime aDateTime = new AMutableDateTime(0L);
@@ -140,6 +149,9 @@
protected ISerializerDeserializer<ANull> nullSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+ protected final AStringSerializerDeserializer untaggedStringSerde =
+ new AStringSerializerDeserializer(new UTF8StringWriter(), new UTF8StringReader());
+
protected final HexParser hexParser = new HexParser();
protected final Base64Parser base64Parser = new Base64Parser();
@@ -150,6 +162,9 @@
protected ISerializerDeserializer<AUUID> uuidSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AUUID);
+ protected ISerializerDeserializer<AGeometry> geomSerde =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+
// To avoid race conditions, the serdes for temporal and spatial data types needs to be one per parser
// ^^^^^^^^^^^^^^^^^^^^^^^^ ??? then why all these serdes are static?
@SuppressWarnings("unchecked")
@@ -195,14 +210,14 @@
this.filename = filename;
}
- protected void parseTime(String time, DataOutput out) throws HyracksDataException {
- int chrononTimeInMs = ATimeParserFactory.parseTimePart(time, 0, time.length());
+ protected void parseTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+ int chrononTimeInMs = ATimeParserFactory.parseTimePart(buffer, begin, len);
aTime.setValue(chrononTimeInMs);
timeSerde.serialize(aTime, out);
}
- protected void parseDate(String date, DataOutput out) throws HyracksDataException {
- long chrononTimeInMs = ADateParserFactory.parseDatePart(date, 0, date.length());
+ protected void parseDate(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+ long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, len);
short temp = 0;
if (chrononTimeInMs < 0 && chrononTimeInMs % GregorianCalendarSystem.CHRONON_OF_DAY != 0) {
temp = 1;
@@ -211,84 +226,93 @@
dateSerde.serialize(aDate, out);
}
- protected void parseDateTime(String datetime, DataOutput out) throws HyracksDataException {
+ protected void parseDateTime(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
// +1 if it is negative (-)
- short timeOffset = (short) ((datetime.charAt(0) == '-') ? 1 : 0);
- timeOffset += 8;
+ int timeOffset = (buffer[begin] == '-') ? 1 : 0;
- if (datetime.charAt(timeOffset) != 'T') {
+ timeOffset = timeOffset + 8 + begin;
+
+ if (buffer[timeOffset] != 'T') {
timeOffset += 2;
- if (datetime.charAt(timeOffset) != 'T') {
- throw new HyracksDataException("This can not be an instance of datetime: missing T");
+ if (buffer[timeOffset] != 'T') {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_INTERVAL_INVALID_DATETIME);
}
}
- long chrononTimeInMs = ADateParserFactory.parseDatePart(datetime, 0, timeOffset);
- chrononTimeInMs +=
- ATimeParserFactory.parseTimePart(datetime, timeOffset + 1, datetime.length() - timeOffset - 1);
+ long chrononTimeInMs = ADateParserFactory.parseDatePart(buffer, begin, timeOffset - begin);
+ chrononTimeInMs += ATimeParserFactory.parseTimePart(buffer, timeOffset + 1, begin + len - timeOffset - 1);
aDateTime.setValue(chrononTimeInMs);
datetimeSerde.serialize(aDateTime, out);
}
- protected void parseDuration(String duration, DataOutput out) throws HyracksDataException {
- ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
+ protected void parseDuration(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
+ ADurationParserFactory.parseDuration(buffer, begin, len, aDuration, ADurationParseOption.All);
durationSerde.serialize(aDuration, out);
}
- protected void parseDateTimeDuration(String durationString, DataOutput out) throws HyracksDataException {
- ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
- ADurationParseOption.All);
+ protected void parseDateTimeDuration(char[] buffer, int begin, int len, DataOutput out)
+ throws HyracksDataException {
+ ADurationParserFactory.parseDuration(buffer, begin, len, aDayTimeDuration, ADurationParseOption.All);
dayTimeDurationSerde.serialize(aDayTimeDuration, out);
}
- protected void parseYearMonthDuration(String durationString, DataOutput out) throws HyracksDataException {
- ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
- ADurationParseOption.All);
+ protected void parseYearMonthDuration(char[] buffer, int begin, int len, DataOutput out)
+ throws HyracksDataException {
+ ADurationParserFactory.parseDuration(buffer, begin, len, aYearMonthDuration, ADurationParseOption.All);
yearMonthDurationSerde.serialize(aYearMonthDuration, out);
}
- protected void parsePoint(String point, DataOutput out) throws HyracksDataException {
+ protected void parsePoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
try {
- aPoint.setValue(Double.parseDouble(point.substring(0, point.indexOf(','))),
- Double.parseDouble(point.substring(point.indexOf(',') + 1, point.length())));
+ int commaIndex = indexOf(buffer, begin, len, ',');
+ aPoint.setValue(parseDouble(buffer, begin, commaIndex - begin),
+ parseDouble(buffer, commaIndex + 1, begin + len - commaIndex - 1));
pointSerde.serialize(aPoint, out);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(point + " can not be an instance of point");
+ } catch (Exception e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+ "point");
}
}
- protected void parse3DPoint(String point3d, DataOutput out) throws HyracksDataException {
+ protected void parse3DPoint(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
try {
- int firstCommaIndex = point3d.indexOf(',');
- int secondCommaIndex = point3d.indexOf(',', firstCommaIndex + 1);
- aPoint3D.setValue(Double.parseDouble(point3d.substring(0, firstCommaIndex)),
- Double.parseDouble(point3d.substring(firstCommaIndex + 1, secondCommaIndex)),
- Double.parseDouble(point3d.substring(secondCommaIndex + 1, point3d.length())));
+ int firstCommaIndex = indexOf(buffer, begin, len, ',');
+ int secondCommaIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ',');
+ aPoint3D.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+ parseDouble(buffer, firstCommaIndex + 1, secondCommaIndex - firstCommaIndex - 1),
+ parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
point3DSerde.serialize(aPoint3D, out);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(point3d + " can not be an instance of point3d");
+ } catch (Exception e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+ "point3d");
}
}
- protected void parseCircle(String circle, DataOutput out) throws HyracksDataException {
+ protected void parseCircle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
try {
- String[] parts = circle.split(" ");
- aPoint.setValue(Double.parseDouble(parts[0].split(",")[0]), Double.parseDouble(parts[0].split(",")[1]));
- aCircle.setValue(aPoint, Double.parseDouble(parts[1].substring(0, parts[1].length())));
+ int firstCommaIndex = indexOf(buffer, begin, len, ',');
+ int spaceIndex = indexOf(buffer, firstCommaIndex + 1, begin + len - firstCommaIndex - 1, ' ');
+ aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+ parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+ aCircle.setValue(aPoint, parseDouble(buffer, spaceIndex + 1, begin + len - spaceIndex - 1));
circleSerde.serialize(aCircle, out);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(circle + " can not be an instance of circle");
+ } catch (Exception e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+ "circle");
}
}
- protected void parseRectangle(String rectangle, DataOutput out) throws HyracksDataException {
+ protected void parseRectangle(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
try {
- String[] points = rectangle.split(" ");
- if (points.length != 2) {
- throw new HyracksDataException("rectangle consists of only 2 points.");
- }
- aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
- aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+ int spaceIndex = indexOf(buffer, begin, len, ' ');
+
+ int firstCommaIndex = indexOf(buffer, begin, len, ',');
+ aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+ parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+
+ int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+ aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+ parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
if (aPoint.getX() > aPoint2.getX() && aPoint.getY() > aPoint2.getY()) {
aRectangle.setValue(aPoint2, aPoint);
} else if (aPoint.getX() < aPoint2.getX() && aPoint.getY() < aPoint2.getY()) {
@@ -298,23 +322,26 @@
"Rectangle arugment must be either (bottom left point, top right point) or (top right point, bottom left point)");
}
rectangleSerde.serialize(aRectangle, out);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(rectangle + " can not be an instance of rectangle");
+ } catch (Exception e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+ "rectangle");
}
}
- protected void parseLine(String line, DataOutput out) throws HyracksDataException {
+ protected void parseLine(char[] buffer, int begin, int len, DataOutput out) throws HyracksDataException {
try {
- String[] points = line.split(" ");
- if (points.length != 2) {
- throw new HyracksDataException("line consists of only 2 points.");
- }
- aPoint.setValue(Double.parseDouble(points[0].split(",")[0]), Double.parseDouble(points[0].split(",")[1]));
- aPoint2.setValue(Double.parseDouble(points[1].split(",")[0]), Double.parseDouble(points[1].split(",")[1]));
+ int spaceIndex = indexOf(buffer, begin, len, ' ');
+ int firstCommaIndex = indexOf(buffer, begin, len, ',');
+ aPoint.setValue(parseDouble(buffer, begin, firstCommaIndex - begin),
+ parseDouble(buffer, firstCommaIndex + 1, spaceIndex - firstCommaIndex - 1));
+ int secondCommaIndex = indexOf(buffer, spaceIndex + 1, begin + len - spaceIndex - 1, ',');
+ aPoint2.setValue(parseDouble(buffer, spaceIndex + 1, secondCommaIndex - spaceIndex - 1),
+ parseDouble(buffer, secondCommaIndex + 1, begin + len - secondCommaIndex - 1));
aLine.setValue(aPoint, aPoint2);
lineSerde.serialize(aLine, out);
- } catch (HyracksDataException e) {
- throw new HyracksDataException(line + " can not be an instance of line");
+ } catch (Exception e) {
+ throw new ParseException(ErrorCode.PARSER_ADM_DATA_PARSER_WRONG_INSTANCE, e, new String(buffer, begin, len),
+ "line");
}
}
@@ -357,4 +384,34 @@
return ATimeParserFactory.parseTimePart(interval, startOffset, endOffset - startOffset + 1);
}
-}
+
+ protected double parseDouble(char[] buffer, int begin, int len) {
+ // TODO: parse double directly from char[]
+ String str = new String(buffer, begin, len);
+ return Double.valueOf(str);
+ }
+
+ protected float parseFloat(char[] buffer, int begin, int len) {
+ //TODO: pares float directly from char[]
+ String str = new String(buffer, begin, len);
+ return Float.valueOf(str);
+ }
+
+ protected int indexOf(char[] buffer, int begin, int len, char target) {
+ for (int i = begin; i < begin + len; i++) {
+ if (buffer[i] == target) {
+ return i;
+ }
+ }
+ throw new IllegalArgumentException("Cannot find " + target + " in " + new String(buffer, begin, len));
+ }
+
+ protected void parseString(char[] buffer, int begin, int length, DataOutput out) throws HyracksDataException {
+ try {
+ out.writeByte(ATypeTag.STRING.serialize());
+ untaggedStringSerde.serialize(buffer, begin, length, out);
+ } catch (IOException e) {
+ throw new ParseException(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
index ce8780d..209ba34 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
@@ -426,26 +426,27 @@
* @throws IOException
*/
private void serializeString(ATypeTag stringVariantType, DataOutput out) throws IOException {
- final String stringValue = jsonParser.getText();
+ char[] buffer = jsonParser.getTextCharacters();
+ int begin = jsonParser.getTextOffset();
+ int len = jsonParser.getTextLength();
final ATypeTag typeToUse = stringVariantType == ATypeTag.ANY ? currentToken().getTypeTag() : stringVariantType;
switch (typeToUse) {
case STRING:
- aString.setValue(stringValue);
- stringSerde.serialize(aString, out);
+ parseString(buffer, begin, len, out);
break;
case DATE:
- parseDate(stringValue, out);
+ parseDate(buffer, begin, len, out);
break;
case DATETIME:
- parseDateTime(stringValue, out);
+ parseDateTime(buffer, begin, len, out);
break;
case TIME:
- parseTime(stringValue, out);
+ parseTime(buffer, begin, len, out);
break;
default:
throw new RuntimeDataException(ErrorCode.TYPE_UNSUPPORTED, jsonParser.currentToken().toString());
}
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
new file mode 100644
index 0000000..34e3eef
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.external.parser;
+
+import java.io.Serializable;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ParseException extends HyracksDataException {
+ private static final long serialVersionUID = 1L;
+ private String filename;
+ private int line = -1;
+ private int column = -1;
+
+ public ParseException(String message) {
+ super(message);
+ }
+
+ public ParseException(int errorCode, Serializable... param) {
+ super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
+ }
+
+ public ParseException(int errorCode, Throwable e, Serializable... param) {
+ super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
+ addSuppressed(e);
+ }
+
+ public ParseException(Throwable cause) {
+ super(cause);
+ }
+
+ public ParseException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public ParseException(Throwable cause, String filename, int line, int column) {
+ super(cause);
+ setLocation(filename, line, column);
+ }
+
+ public void setLocation(String filename, int line, int column) {
+ this.filename = filename;
+ this.line = line;
+ this.column = column;
+ }
+
+ @Override
+ public String getMessage() {
+ StringBuilder msg = new StringBuilder("Parse error");
+ if (filename != null) {
+ msg.append(" in file " + filename);
+ }
+ if (line >= 0) {
+ if (column >= 0) {
+ msg.append(" at (" + line + ", " + column + ")");
+ } else {
+ msg.append(" in line " + line);
+ }
+ }
+ return msg.append(": " + super.getMessage()).toString();
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
index f06528e..a4e4c33 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ADMDataParserTest.java
@@ -18,38 +18,54 @@
*/
package org.apache.asterix.external.parser.test;
+import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.asterix.external.parser.ADMDataParser;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AMutableDate;
import org.apache.asterix.om.base.AMutableDateTime;
import org.apache.asterix.om.base.AMutableTime;
+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.hadoop.io.DataInputByteBuffer;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.junit.Assert;
import org.junit.Test;
+import com.esri.core.geometry.ogc.OGCPoint;
+
import junit.extensions.PA;
public class ADMDataParserTest {
@Test
public void test() throws IOException {
- String[] dates = { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" };
+ char[][] dates = toChars(
+ new String[] { "-9537-08-04", "9656-06-03", "-9537-04-04", "9656-06-04", "-9537-10-04", "9626-09-05" });
AMutableDate[] parsedDates =
new AMutableDate[] { new AMutableDate(-4202630), new AMutableDate(2807408), new AMutableDate(-4202752),
new AMutableDate(2807409), new AMutableDate(-4202569), new AMutableDate(2796544), };
- String[] times = { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z", "04:16:42.321Z",
- "12:22:56.816Z" };
+ char[][] times = toChars(new String[] { "12:04:45.689Z", "12:41:59.002Z", "12:10:45.169Z", "15:37:48.736Z",
+ "04:16:42.321Z", "12:22:56.816Z" });
AMutableTime[] parsedTimes =
new AMutableTime[] { new AMutableTime(43485689), new AMutableTime(45719002), new AMutableTime(43845169),
new AMutableTime(56268736), new AMutableTime(15402321), new AMutableTime(44576816), };
- String[] dateTimes = { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
- "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" };
+ char[][] dateTimes = toChars(
+ new String[] { "-2640-10-11T17:32:15.675Z", "4104-02-01T05:59:11.902Z", "0534-12-08T08:20:31.487Z",
+ "6778-02-16T22:40:21.653Z", "2129-12-12T13:18:35.758Z", "8647-07-01T13:10:19.691Z" });
AMutableDateTime[] parsedDateTimes =
new AMutableDateTime[] { new AMutableDateTime(-145452954464325L), new AMutableDateTime(67345192751902L),
new AMutableDateTime(-45286270768513L), new AMutableDateTime(151729886421653L),
@@ -70,24 +86,24 @@
while (round++ < 10000) {
// Test parseDate.
for (int index = 0; index < dates.length; ++index) {
- PA.invokeMethod(parser, "parseDate(java.lang.String, java.io.DataOutput)", dates[index],
- dos);
+ PA.invokeMethod(parser, "parseDate(char[], int, int, java.io.DataOutput)", dates[index],
+ 0, dates[index].length, dos);
AMutableDate aDate = (AMutableDate) PA.getValue(parser, "aDate");
Assert.assertTrue(aDate.equals(parsedDates[index]));
}
// Tests parseTime.
for (int index = 0; index < times.length; ++index) {
- PA.invokeMethod(parser, "parseTime(java.lang.String, java.io.DataOutput)", times[index],
- dos);
+ PA.invokeMethod(parser, "parseTime(char[], int, int, java.io.DataOutput)", times[index],
+ 0, times[index].length, dos);
AMutableTime aTime = (AMutableTime) PA.getValue(parser, "aTime");
Assert.assertTrue(aTime.equals(parsedTimes[index]));
}
// Tests parseDateTime.
for (int index = 0; index < dateTimes.length; ++index) {
- PA.invokeMethod(parser, "parseDateTime(java.lang.String, java.io.DataOutput)",
- dateTimes[index], dos);
+ PA.invokeMethod(parser, "parseDateTime(char[], int, int, java.io.DataOutput)",
+ dateTimes[index], 0, dateTimes[index].length, dos);
AMutableDateTime aDateTime = (AMutableDateTime) PA.getValue(parser, "aDateTime");
Assert.assertTrue(aDateTime.equals(parsedDateTimes[index]));
}
@@ -113,4 +129,58 @@
// Asserts no failure.
Assert.assertTrue(errorCount.get() == 0);
}
-}
+
+ private char[][] toChars(String[] strings) {
+ char[][] results = new char[strings.length][];
+ for (int i = 0; i < strings.length; i++) {
+ results[i] = strings[i].toCharArray();
+ }
+ return results;
+ }
+
+ @Test
+ public void testWKTParser() {
+ try {
+ ARecordType recordType = new ARecordType("POIType", new String[] { "id", "coord" },
+ new IAType[] { BuiltinType.AINT32, BuiltinType.AGEOMETRY }, false);
+
+ String wktObject = "{\"id\": 123, \"coord\": \"POINT(3 4)\"}";
+ InputStream in = new ByteArrayInputStream(wktObject.getBytes());
+ ADMDataParser parser = new ADMDataParser(recordType, true);
+ parser.setInputStream(in);
+ ByteArrayOutputStream baos = new ByteArrayOutputStream();
+ DataOutputStream out = new DataOutputStream(baos);
+ parser.parse(out);
+ out.close();
+ byte[] serialized = baos.toByteArray();
+
+ // Parse to make sure it was correct
+ ByteBuffer bb = ByteBuffer.wrap(serialized);
+ Assert.assertEquals(ATypeTag.SERIALIZED_RECORD_TYPE_TAG, bb.get());
+ Assert.assertEquals(serialized.length, bb.getInt()); // Total record size including header
+ Assert.assertEquals(2, bb.getInt()); // # of records
+ int offsetOfID = bb.getInt();
+ int offsetOfGeometry = bb.getInt();
+ ISerializerDeserializer intDeser =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT32);
+ Assert.assertEquals(offsetOfID, bb.position());
+ // Serialize the two records
+ DataInputByteBuffer dataIn = new DataInputByteBuffer();
+ dataIn.reset(bb);
+ Object o = intDeser.deserialize(dataIn);
+ Assert.assertEquals(new AInt32(123), o);
+ ISerializerDeserializer geomDeser =
+ SerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AGEOMETRY);
+ Object point = geomDeser.deserialize(dataIn);
+ Assert.assertTrue(point instanceof AGeometry);
+ Assert.assertTrue(((AGeometry) point).getGeometry() instanceof OGCPoint);
+ OGCPoint p = (OGCPoint) ((AGeometry) point).getGeometry();
+ Assert.assertEquals(3.0, p.X(), 1E-5);
+ Assert.assertEquals(4.0, p.Y(), 1E-5);
+ } catch (IOException e) {
+ e.printStackTrace();
+ Assert.fail("Error in parsing");
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-geo/pom.xml b/asterixdb/asterix-geo/pom.xml
new file mode 100644
index 0000000..671e57e
--- /dev/null
+++ b/asterixdb/asterix-geo/pom.xml
@@ -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.
+ !-->
+<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>
+ <parent>
+ <artifactId>apache-asterixdb</artifactId>
+ <groupId>org.apache.asterix</groupId>
+ <version>0.9.4-SNAPSHOT</version>
+ </parent>
+ <artifactId>asterix-geo</artifactId>
+
+ <licenses>
+ <license>
+ <name>Apache License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ <comments>A business-friendly OSS license</comments>
+ </license>
+ </licenses>
+
+ <properties>
+ <root.dir>${basedir}/..</root.dir>
+ </properties>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-evaluator-generator-maven-plugin</artifactId>
+ <version>${project.version}</version>
+ <configuration>
+ <evaluatorPackagePrefix>org.apache.asterix.runtime.evaluators</evaluatorPackagePrefix>
+ </configuration>
+ <executions>
+ <execution>
+ <id>generate-evaluator</id>
+ <phase>process-classes</phase>
+ <goals>
+ <goal>generate-evaluator</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ <phase>test-compile</phase>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>validate</phase>
+ <goals>
+ <goal>check</goal>
+ </goals>
+ </execution>
+ </executions>
+ <configuration>
+ <excludes combine.children="append">
+ <exclude>data/**</exclude>
+ </excludes>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-data-std</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-runtime</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-om</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-runtime</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.esri.geometry</groupId>
+ <artifactId>esri-geometry-api</artifactId>
+ <version>2.0.0</version>
+ </dependency>
+ </dependencies>
+
+</project>
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
new file mode 100644
index 0000000..ec45877
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.aggregates;
+
+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.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * NULLs are also counted.
+ */
+public class STUnionAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STUnionAggregateDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_UNION_AGG;
+ }
+
+ @Override
+ public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IAggregateEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IAggregateEvaluator createAggregateEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new STUnionAggregateFunction(args, ctx);
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
new file mode 100644
index 0000000..c9c2eca
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/STUnionAggregateFunction.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.aggregates;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+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.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * STUnion aggregates a set of objects into one object. If the input is a set of overlapping polygons, their union is
+ * computed and returned as a multipolygon. Similarly, if the input is a set of points or linestring, a multipoint or
+ * multilinestring is created. Is the result contains geometries of different types, e.g., points and linestring, the
+ * output is a GeometryCollection.
+ */
+public class STUnionAggregateFunction implements IAggregateEvaluator {
+ /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+ public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+ @SuppressWarnings("unchecked")
+ private ISerializerDeserializer<AGeometry> geometrySerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY);
+ private IPointable inputVal = new VoidPointable();
+ private IScalarEvaluator eval;
+ protected OGCGeometry geometry;
+
+ private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+
+ public STUnionAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context)
+ throws HyracksDataException {
+ eval = args[0].createScalarEvaluator(context);
+ }
+
+ @Override
+ public void init() throws HyracksDataException {
+ // Initialize the resulting geometry with an empty point.
+ geometry = new OGCPoint(new Point(), DEFAULT_CRS);
+ }
+
+ @Override
+ public void step(IFrameTupleReference tuple) throws HyracksDataException {
+ eval.evaluate(tuple, inputVal);
+ byte[] data = inputVal.getByteArray();
+ int offset = inputVal.getStartOffset();
+ int len = inputVal.getLength();
+ ATypeTag typeTag =
+ EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
+ // Ignore SYSTEM_NULL.
+ if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
+ processNull();
+ } else if (typeTag == ATypeTag.GEOMETRY) {
+ DataInput dataIn = new DataInputStream(new ByteArrayInputStream(data, offset + 1, len - 1));
+ OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+ geometry = geometry.union(geometry1);
+ }
+ }
+
+ @Override
+ public void finish(IPointable resultPointable) throws HyracksDataException {
+ resultStorage.reset();
+ try {
+ geometrySerde.serialize(new AGeometry(geometry), resultStorage.getDataOutput());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ resultPointable.set(resultStorage);
+ }
+
+ @Override
+ public void finishPartial(IPointable resultPointable) throws HyracksDataException {
+ finish(resultPointable);
+ }
+
+ protected void processNull() throws UnsupportedItemTypeException {
+ throw new UnsupportedItemTypeException(BuiltinFunctions.ST_UNION, ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.java
new file mode 100644
index 0000000..96b6478
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/aggregates/ScalarSTUnionAggregateDescriptor.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.geo.aggregates;
+
+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.aggregates.scalar.AbstractScalarAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSTUnionAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public final static FunctionIdentifier FID = BuiltinFunctions.SCALAR_ST_UNION_AGG;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ScalarSTUnionAggregateDescriptor(STUnionAggregateDescriptor.FACTORY.createFunctionDescriptor());
+ }
+ };
+
+ private ScalarSTUnionAggregateDescriptor(IFunctionDescriptor aggFuncDesc) {
+ super(aggFuncDesc);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return FID;
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
new file mode 100644
index 0000000..1644b99
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators;
+
+import org.apache.asterix.geo.aggregates.STUnionAggregateDescriptor;
+import org.apache.asterix.geo.aggregates.ScalarSTUnionAggregateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.ParseGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAreaDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsBinaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsGeoJSONDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STAsTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STBoundaryDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STContainsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCoordDimDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STCrossesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDimensionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDisjointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STDistanceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEndPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEnvelopeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STEqualsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STExteriorRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromTextSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomFromWKBDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeomentryTypeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STGeometryNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STInteriorRingNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIntersectsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsClosedDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsCollectionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsEmptyDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsRingDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STIsSimpleDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLengthDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STLineFromMultiPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakeEnvelopeDescriptorSRID;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePoint3DWithMDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMakePointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNPointsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumGeometriesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STNumInteriorRingsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STOverlapsDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPointNDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STPolygonizeDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STRelateDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSRIDDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STStartPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STSymDifferenceDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STTouchesDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STUnionDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STWithinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STXMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STYMinDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMaxDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STZMinDescriptor;
+import org.apache.asterix.om.functions.IFunctionCollection;
+import org.apache.asterix.om.functions.IFunctionRegistrant;
+
+public class GeoFunctionRegistrant implements IFunctionRegistrant {
+ @Override
+ public void register(IFunctionCollection fc) {
+ //Geo functions
+ fc.add(ScalarSTUnionAggregateDescriptor.FACTORY);
+ fc.add(STUnionAggregateDescriptor.FACTORY);
+
+ //GeoJSON
+ fc.add(ParseGeoJSONDescriptor.FACTORY);
+ fc.add(STAreaDescriptor.FACTORY);
+ fc.add(STMakePointDescriptor.FACTORY);
+ fc.add(STMakePoint3DDescriptor.FACTORY);
+ fc.add(STMakePoint3DWithMDescriptor.FACTORY);
+ fc.add(STIntersectsDescriptor.FACTORY);
+ fc.add(STUnionDescriptor.FACTORY);
+ fc.add(STIsCollectionDescriptor.FACTORY);
+ fc.add(STContainsDescriptor.FACTORY);
+ fc.add(STCrossesDescriptor.FACTORY);
+ fc.add(STDisjointDescriptor.FACTORY);
+ fc.add(STEqualsDescriptor.FACTORY);
+ fc.add(STOverlapsDescriptor.FACTORY);
+ fc.add(STTouchesDescriptor.FACTORY);
+ fc.add(STWithinDescriptor.FACTORY);
+ fc.add(STIsEmptyDescriptor.FACTORY);
+ fc.add(STIsSimpleDescriptor.FACTORY);
+ fc.add(STCoordDimDescriptor.FACTORY);
+ fc.add(STDimensionDescriptor.FACTORY);
+ fc.add(STGeomentryTypeDescriptor.FACTORY);
+ fc.add(STMDescriptor.FACTORY);
+ fc.add(STNRingsDescriptor.FACTORY);
+ fc.add(STNPointsDescriptor.FACTORY);
+ fc.add(STNumGeometriesDescriptor.FACTORY);
+ fc.add(STNumInteriorRingsDescriptor.FACTORY);
+ fc.add(STSRIDDescriptor.FACTORY);
+ fc.add(STXDescriptor.FACTORY);
+ fc.add(STYDescriptor.FACTORY);
+ fc.add(STXMaxDescriptor.FACTORY);
+ fc.add(STXMinDescriptor.FACTORY);
+ fc.add(STYMaxDescriptor.FACTORY);
+ fc.add(STYMinDescriptor.FACTORY);
+ fc.add(STZDescriptor.FACTORY);
+ fc.add(STZMaxDescriptor.FACTORY);
+ fc.add(STZMinDescriptor.FACTORY);
+ fc.add(STAsBinaryDescriptor.FACTORY);
+ fc.add(STAsTextDescriptor.FACTORY);
+ fc.add(STAsGeoJSONDescriptor.FACTORY);
+ fc.add(STDistanceDescriptor.FACTORY);
+ fc.add(STLengthDescriptor.FACTORY);
+ fc.add(STGeomFromTextDescriptor.FACTORY);
+ fc.add(STGeomFromTextSRIDDescriptor.FACTORY);
+ fc.add(STGeomFromWKBDescriptor.FACTORY);
+ fc.add(STLineFromMultiPointDescriptor.FACTORY);
+ fc.add(STMakeEnvelopeDescriptorSRID.FACTORY);
+ fc.add(STIsClosedDescriptor.FACTORY);
+ fc.add(STIsRingDescriptor.FACTORY);
+ fc.add(STRelateDescriptor.FACTORY);
+ fc.add(STBoundaryDescriptor.FACTORY);
+ fc.add(STEndPointDescriptor.FACTORY);
+ fc.add(STEnvelopeDescriptor.FACTORY);
+ fc.add(STExteriorRingDescriptor.FACTORY);
+ fc.add(STGeometryNDescriptor.FACTORY);
+ fc.add(STInteriorRingNDescriptor.FACTORY);
+ fc.add(STPointNDescriptor.FACTORY);
+ fc.add(STStartPointDescriptor.FACTORY);
+ fc.add(STDifferenceDescriptor.FACTORY);
+ fc.add(STIntersectionDescriptor.FACTORY);
+ fc.add(STSymDifferenceDescriptor.FACTORY);
+ fc.add(STPolygonizeDescriptor.FACTORY);
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.java
new file mode 100644
index 0000000..f609beb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionTypeInferers.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.geo.evaluators;
+
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+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.exceptions.NotImplementedException;
+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.expressions.IVariableTypeEnvironment;
+
+public class GeoFunctionTypeInferers {
+ private GeoFunctionTypeInferers() {
+ }
+
+ public static final class GeometryConstructorTypeInferer implements IFunctionTypeInferer {
+ @Override
+ public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+ CompilerProperties compilerProps) throws AlgebricksException {
+ AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+ IAType t = (IAType) context.getType(fce.getArguments().get(0).getValue());
+ ATypeTag typeTag = t.getTypeTag();
+ if (typeTag.equals(ATypeTag.OBJECT)) {
+ fd.setImmutableStates(t);
+ } else if (typeTag.equals(ATypeTag.ANY)) {
+ fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE);
+ } else {
+ throw new NotImplementedException("parse-geojson for data of type " + t);
+ }
+ }
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.java
new file mode 100644
index 0000000..325b10a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractGetValDescriptor.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.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+
+public abstract class AbstractGetValDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ public double getVal(byte[] bytes, int offset) throws TypeMismatchException {
+ if (bytes[offset] == ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG) {
+ return ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
+ } else if (bytes[offset] == ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ return AInt64SerializerDeserializer.getLong(bytes, offset + 1);
+ } else {
+ throw new TypeMismatchException(getIdentifier(), 1, bytes[offset], ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG,
+ ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ }
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
new file mode 100644
index 0000000..83925a1
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTDoubleGeometryDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTDoubleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ abstract protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1)
+ throws HyracksDataException;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractSTDoubleGeometryEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class AbstractSTDoubleGeometryEvaluator implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+ private final IPointable argPtr0;
+ private final IPointable argPtr1;
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+
+ public AbstractSTDoubleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ argPtr0 = new VoidPointable();
+ argPtr1 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ eval1 = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ eval0.evaluate(tuple, argPtr0);
+ eval1.evaluate(tuple, argPtr1);
+
+ try {
+ byte[] bytes0 = argPtr0.getByteArray();
+ int offset0 = argPtr0.getStartOffset();
+ int len0 = argPtr0.getLength();
+ byte[] bytes1 = argPtr1.getByteArray();
+ int offset1 = argPtr1.getStartOffset();
+ int len1 = argPtr1.getLength();
+
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ if (tag != ATypeTag.GEOMETRY) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ if (tag != ATypeTag.GEOMETRY) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes1[offset1],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+ OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+ DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+ OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+ Object finalResult = evaluateOGCGeometry(geometry0, geometry1);
+ if (finalResult instanceof OGCGeometry) {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+ } else if (finalResult instanceof Boolean) {
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+ .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+ } else if (finalResult instanceof Double) {
+ out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ out.writeDouble((double) finalResult);
+ }
+
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
new file mode 100644
index 0000000..62e8db9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTGeometryNDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTGeometryNDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ private static final long serialVersionUID = 1L;
+
+ abstract protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new AbstractSTGeometryNEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class AbstractSTGeometryNEvaluator implements IScalarEvaluator {
+
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+ private IPointable inputArg0;
+ private IScalarEvaluator eval0;
+
+ public AbstractSTGeometryNEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[0].createScalarEvaluator(ctx);
+ inputArg0 = new VoidPointable();
+ eval0 = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] data = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ eval0.evaluate(tuple, inputArg0);
+ byte[] data0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+
+ if (data[offset] != ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, data0[offset0], ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ }
+
+ ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+ int n = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+
+ OGCGeometry geometryN = evaluateOGCGeometry(geometry, n);
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry(geometryN), out);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
new file mode 100644
index 0000000..0acf731
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABinary;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.AMutableInt32;
+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.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.util.string.UTF8StringUtil;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public abstract class AbstractSTSingleGeometryDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ abstract protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException;
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractSTSingleGeometryEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class AbstractSTSingleGeometryEvaluator implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+ private final IPointable argPtr0;
+ private final IScalarEvaluator eval0;
+
+ private final AMutableInt32 intRes;
+
+ public AbstractSTSingleGeometryEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ argPtr0 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ intRes = new AMutableInt32(0);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+ eval0.evaluate(tuple, argPtr0);
+
+ try {
+ byte[] bytes0 = argPtr0.getByteArray();
+ int offset0 = argPtr0.getStartOffset();
+ int len0 = argPtr0.getLength();
+
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ if (tag != ATypeTag.GEOMETRY) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+ OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+
+ Object finalResult = evaluateOGCGeometry(geometry0);
+ if (finalResult == null) {
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ } else if (finalResult instanceof Double) {
+ out.writeByte(ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+ out.writeDouble((double) finalResult);
+ } else if (finalResult instanceof Boolean) {
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+ .serialize((boolean) finalResult ? ABoolean.TRUE : ABoolean.FALSE, out);
+ } else if (finalResult instanceof Integer) {
+ intRes.setValue((int) finalResult);
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32)
+ .serialize(intRes, out);
+ } else if (finalResult instanceof String) {
+ out.write(ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ out.write(UTF8StringUtil.writeStringToBytes((String) finalResult));
+ } else if (finalResult instanceof byte[]) {
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABINARY)
+ .serialize(new ABinary((byte[]) finalResult), out);
+ } else if (finalResult instanceof OGCGeometry) {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+ }
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
new file mode 100644
index 0000000..e59c3f8
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/ParseGeoJSONDescriptor.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.MapOGCStructure;
+import com.esri.core.geometry.OperatorImportFromGeoJson;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
+import org.apache.asterix.geo.evaluators.GeoFunctionTypeInferers;
+import org.apache.asterix.om.base.AOrderedList;
+import org.apache.asterix.om.base.ARecord;
+import org.apache.asterix.om.base.IAObject;
+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.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class ParseGeoJSONDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new ParseGeoJSONDescriptor();
+ }
+
+ @Override
+ public IFunctionTypeInferer createFunctionTypeInferer() {
+ return new GeoFunctionTypeInferers.GeometryConstructorTypeInferer();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+ private ARecordType recType;
+
+ @Override
+ public void setImmutableStates(Object... states) {
+ this.recType = (ARecordType) states[0];
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GEOMETRY_CONSTRUCTOR;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new ParseGeoJSONEvaluator(args[0], ctx);
+ }
+ };
+ }
+
+ private class ParseGeoJSONEvaluator implements IScalarEvaluator {
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+ private OperatorImportFromGeoJson geoJsonImporter;
+
+ public ParseGeoJSONEvaluator(IScalarEvaluatorFactory factory, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = factory.createScalarEvaluator(ctx);
+ geoJsonImporter = OperatorImportFromGeoJson.local();
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] data = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ if (data[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+ throw new TypeMismatchException(BuiltinFunctions.GEOMETRY_CONSTRUCTOR, 0, data[offset],
+ ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
+ }
+ ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+ DataInput dataIn = new DataInputStream(inStream);
+ try {
+ String geometry = recordToString(new ARecordSerializerDeserializer(recType).deserialize(dataIn));
+ MapOGCStructure structure = geoJsonImporter.executeOGC(0, geometry, null);
+ OGCGeometry ogcGeometry =
+ OGCGeometry.createFromOGCStructure(structure.m_ogcStructure, structure.m_spatialReference);
+ ByteBuffer buffer = ogcGeometry.asBinary();
+ byte[] wKBGeometryBuffer = buffer.array();
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ out.writeInt(wKBGeometryBuffer.length);
+ out.write(wKBGeometryBuffer);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ }
+
+ public String recordToString(ARecord record) {
+ StringBuilder sb = new StringBuilder();
+ sb.append("{ ");
+ String[] fieldNames = record.getType().getFieldNames();
+ IAObject val;
+ if (fieldNames != null) {
+ for (int i = 0; i < fieldNames.length; i++) {
+ if (i > 0) {
+ sb.append(", ");
+ }
+ sb.append("\"").append(fieldNames[i]).append("\"").append(": ");
+ val = record.getValueByPos(i);
+ if (val instanceof ARecord) {
+ sb.append(recordToString((ARecord) val));
+ } else if (val instanceof AOrderedList) {
+ sb.append(listToString((AOrderedList) val));
+ } else {
+ sb.append(val);
+ }
+ }
+ }
+ sb.append(" }");
+ return sb.toString();
+ }
+
+ public String listToString(AOrderedList list) {
+ StringBuilder sb = new StringBuilder();
+ boolean first = true;
+ IAObject val;
+ sb.append("[ ");
+ for (int i = 0; i < list.size(); i++) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(", ");
+ }
+ val = list.getItem(i);
+ if (val instanceof ARecord) {
+ sb.append(recordToString((ARecord) val));
+ } else if (val instanceof AOrderedList) {
+ sb.append(listToString((AOrderedList) val));
+ } else {
+ sb.append(val);
+ }
+ }
+ sb.append(" ]");
+ return sb.toString();
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
new file mode 100644
index 0000000..caf0b89
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAreaDescriptor.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAreaDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STAreaDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ double area;
+ if (!"GeometryCollection".equals(geometry.geometryType())) {
+ area = geometry.getEsriGeometry().calculateArea2D();
+ } else {
+ GeometryCursor cursor = geometry.getEsriGeometryCursor();
+ Geometry geometry1 = cursor.next();
+ area = 0;
+ while (geometry1 != null) {
+ area += geometry1.calculateArea2D();
+ geometry1 = cursor.next();
+ }
+ }
+ return area;
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_AREA;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.java
new file mode 100644
index 0000000..387d07c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsBinaryDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsBinaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STAsBinaryDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.asBinary().array();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_AS_BINARY;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.java
new file mode 100644
index 0000000..b12b851
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsGeoJSONDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsGeoJSONDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STAsGeoJSONDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.asGeoJson();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_AS_GEOJSON;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.java
new file mode 100644
index 0000000..3e310d9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STAsTextDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STAsTextDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STAsTextDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.asText();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_AS_TEXT;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.java
new file mode 100644
index 0000000..55c5212
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STBoundaryDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STBoundaryDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STBoundaryDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.boundary();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_BOUNDARY;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.java
new file mode 100644
index 0000000..f9f692f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STContainsDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STContainsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STContainsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.contains(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_CONTAINS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.java
new file mode 100644
index 0000000..d1a3efd
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCoordDimDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCoordDimDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STCoordDimDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.coordinateDimension();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_COORD_DIM;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.java
new file mode 100644
index 0000000..1ac67fe
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STCrossesDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STCrossesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STCrossesDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.crosses(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_CROSSES;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
new file mode 100644
index 0000000..a0209f9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDifferenceDescriptor.java
@@ -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.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STDifferenceDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.difference(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_DIFFERENCE;
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.java
new file mode 100644
index 0000000..492d045
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDimensionDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDimensionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STDimensionDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.dimension();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_DIMENSION;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.java
new file mode 100644
index 0000000..17272ce
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDisjointDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDisjointDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STDisjointDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.disjoint(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_DISJOINT;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.java
new file mode 100644
index 0000000..50cb129
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STDistanceDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STDistanceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STDistanceDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.distance(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_DISTANCE;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.java
new file mode 100644
index 0000000..6b8e43b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEndPointDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEndPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STEndPointDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCCurve) {
+ return ((OGCCurve) geometry).endPoint();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_END_POINT;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.java
new file mode 100644
index 0000000..f5d7d0d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEnvelopeDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEnvelopeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STEnvelopeDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.envelope();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_ENVELOPE;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.java
new file mode 100644
index 0000000..66c2ab1
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STEqualsDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STEqualsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STEqualsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.equals(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_EQUALS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.java
new file mode 100644
index 0000000..bda2c90
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STExteriorRingDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STExteriorRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STExteriorRingDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPolygon) {
+ return ((OGCPolygon) geometry).exteriorRing();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_EXTERIOR_RING;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
new file mode 100644
index 0000000..8676600
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextDescriptor.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+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.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STGeomFromTextDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_GEOM_FROM_TEXT;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new STGeomFromTextEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class STGeomFromTextEvaluator implements IScalarEvaluator {
+
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+ private OperatorImportFromWkt wktImporter;
+
+ public STGeomFromTextEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[0].createScalarEvaluator(ctx);
+ wktImporter = OperatorImportFromWkt.local();
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] data = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(BuiltinFunctions.ST_GEOM_FROM_TEXT, 0, data[offset],
+ ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ try {
+ String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+ OGCStructure structure;
+
+ structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+ OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+ ByteBuffer buffer = ogcGeometry.asBinary();
+ byte[] wKBGeometryBuffer = buffer.array();
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ out.writeInt(wKBGeometryBuffer.length);
+ out.write(wKBGeometryBuffer);
+ result.set(resultStorage);
+
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
new file mode 100644
index 0000000..01f7438
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromTextSRIDDescriptor.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+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.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STGeomFromTextSRIDDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STGeomFromTextSRIDDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_GEOM_FROM_TEXT_SRID;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new STGeomFromTextSRIDEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class STGeomFromTextSRIDEvaluator implements IScalarEvaluator {
+
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+ private IPointable inputArg0;
+ private IScalarEvaluator eval0;
+ private OperatorImportFromWkt wktImporter;
+
+ public STGeomFromTextSRIDEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[0].createScalarEvaluator(ctx);
+ inputArg0 = new VoidPointable();
+ eval0 = args[1].createScalarEvaluator(ctx);
+ wktImporter = OperatorImportFromWkt.local();
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] data = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ eval0.evaluate(tuple, inputArg0);
+ byte[] data0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+
+ if (data[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, data[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ if (data0[offset0] != ATypeTag.SERIALIZED_INT64_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, data0[offset0], ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+ }
+
+ ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ try {
+ String geometry = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+ int srid = (int) AInt64SerializerDeserializer.getLong(data0, offset0 + 1);
+ OGCStructure structure;
+
+ structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, geometry, null);
+ OGCGeometry ogcGeometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(srid));
+ ByteBuffer buffer = ogcGeometry.asBinary();
+ byte[] wKBGeometryBuffer = buffer.array();
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ out.writeInt(wKBGeometryBuffer.length);
+ out.write(wKBGeometryBuffer);
+ result.set(resultStorage);
+
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
new file mode 100644
index 0000000..ad3365e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomFromWKBDescriptor.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+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.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STGeomFromWKBDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STGeomFromWKBDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_GEOM_FROM_WKB;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new STGeomFromWKBEvaulator(args, ctx);
+ }
+ };
+ }
+
+ private class STGeomFromWKBEvaulator implements IScalarEvaluator {
+
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+
+ public STGeomFromWKBEvaulator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[0].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] data = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ if (data[offset] != ATypeTag.SERIALIZED_BINARY_TYPE_TAG) {
+ throw new TypeMismatchException(BuiltinFunctions.ST_GEOM_FROM_WKB, 0, data[offset],
+ ATypeTag.SERIALIZED_BINARY_TYPE_TAG);
+ }
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ ByteArrayInputStream inStream = new ByteArrayInputStream(data, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ data = ByteArraySerializerDeserializer.read(dataIn);
+ out.writeInt(data.length);
+ out.write(data);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.java
new file mode 100644
index 0000000..8e7ae39
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeomentryTypeDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeomentryTypeDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STGeomentryTypeDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.geometryType();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.GEOMETRY_TYPE;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.java
new file mode 100644
index 0000000..9bf3699
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STGeometryNDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STGeometryNDescriptor extends AbstractSTGeometryNDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STGeometryNDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_GEOMETRY_N;
+ }
+
+ @Override
+ protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+ if (geometry instanceof OGCGeometryCollection) {
+ return ((OGCGeometryCollection) geometry).geometryN(n);
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.java
new file mode 100644
index 0000000..25df415
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STInteriorRingNDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STInteriorRingNDescriptor extends AbstractSTGeometryNDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STInteriorRingNDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_INTERIOR_RING_N;
+ }
+
+ @Override
+ protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+ if (geometry instanceof OGCPolygon) {
+ return ((OGCPolygon) geometry).interiorRingN(n);
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
new file mode 100644
index 0000000..6f3314e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectionDescriptor.java
@@ -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.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIntersectionDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.intersection(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_INTERSECTION;
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.java
new file mode 100644
index 0000000..1a4f37f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIntersectsDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIntersectsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIntersectsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.intersects(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_INTERSECTS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
new file mode 100644
index 0000000..59edb4e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsClosedDescriptor.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import com.esri.core.geometry.ogc.OGCMultiCurve;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPoint;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsClosedDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIsClosedDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return isClosed(geometry);
+ }
+
+ private boolean isClosed(OGCGeometry geometry) {
+ if (geometry instanceof OGCCurve) {
+ return ((OGCCurve) geometry).isClosed();
+ } else if (geometry instanceof OGCMultiCurve) {
+ return ((OGCMultiCurve) geometry).isClosed();
+ } else if (geometry instanceof OGCMultiPoint || geometry instanceof OGCMultiPolygon
+ || geometry instanceof OGCPolygon || geometry instanceof OGCPoint) {
+ return true;
+ } else if (geometry instanceof OGCGeometryCollection) {
+ OGCGeometryCollection geometryCollection = (OGCGeometryCollection) geometry;
+ int num = geometryCollection.numGeometries();
+ for (int i = 0; i < num; ++i) {
+ if (!isClosed(geometryCollection.geometryN(i))) {
+ return false;
+ }
+ }
+ return true;
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_IS_CLOSED;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
new file mode 100644
index 0000000..92767e9
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsCollectionDescriptor.java
@@ -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.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsCollectionDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIsCollectionDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ String type = geometry.geometryType();
+ return "GeometryCollection".equals(type) || "MultiLineString".equals(type) || "MultiPoint".equals(type)
+ || "MultiPolygon".equals(type);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_IS_COLLECTION;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.java
new file mode 100644
index 0000000..acf20b7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsEmptyDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsEmptyDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIsEmptyDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.isEmpty();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_IS_EMPTY;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.java
new file mode 100644
index 0000000..3c9375a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsRingDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsRingDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIsRingDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCCurve) {
+ return ((OGCCurve) geometry).isRing();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_IS_RING;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.java
new file mode 100644
index 0000000..fdb948f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STIsSimpleDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STIsSimpleDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STIsSimpleDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.isSimple();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_IS_SIMPLE;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
new file mode 100644
index 0000000..65fd085
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLengthDescriptor.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLengthDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STLengthDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCLineString) {
+ return geometry.getEsriGeometry().calculateLength2D();
+ } else if (geometry instanceof OGCMultiLineString) {
+ GeometryCursor cursor = geometry.getEsriGeometryCursor();
+ double length = 0;
+ Geometry geometry1 = cursor.next();
+ while (geometry1 != null) {
+ length += geometry1.calculateLength2D();
+ geometry1 = cursor.next();
+ }
+ return length;
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_LENGTH;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.java
new file mode 100644
index 0000000..c783fe0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STLineFromMultiPointDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.Polyline;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import com.esri.core.geometry.ogc.OGCMultiPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STLineFromMultiPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STLineFromMultiPointDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCMultiPoint) {
+ Polyline polyline = new Polyline();
+ OGCMultiPoint multiPoint = (OGCMultiPoint) geometry;
+ int numPoints = multiPoint.numGeometries();
+ polyline.startPath((Point) multiPoint.geometryN(0).getEsriGeometry());
+ for (int i = 1; i < numPoints; i++) {
+ polyline.lineTo((Point) multiPoint.geometryN(i).getEsriGeometry());
+ }
+ return new OGCLineString(polyline, 0, SpatialReference.create(4326));
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_LINE_FROM_MULTIPOINT;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.java
new file mode 100644
index 0000000..46ce0fb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STMDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPoint) {
+ return ((OGCPoint) geometry).M();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_M;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
new file mode 100644
index 0000000..efb4032
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCGeometry;
+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.exceptions.InvalidDataFormatException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public class STMakeEnvelopeDescriptorSRID extends AbstractGetValDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMakeEnvelopeDescriptorSRID();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MAKE_ENVELOPE;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new STMakeEnvelopeEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class STMakeEnvelopeEvaluator implements IScalarEvaluator {
+
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg0;
+ private IScalarEvaluator eval0;
+ private IPointable inputArg1;
+ private IScalarEvaluator eval1;
+ private IPointable inputArg2;
+ private IScalarEvaluator eval2;
+ private IPointable inputArg3;
+ private IScalarEvaluator eval3;
+ private IPointable inputArg4;
+ private IScalarEvaluator eval4;
+
+ public STMakeEnvelopeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg0 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ inputArg1 = new VoidPointable();
+ eval1 = args[1].createScalarEvaluator(ctx);
+ inputArg2 = new VoidPointable();
+ eval2 = args[2].createScalarEvaluator(ctx);
+ inputArg3 = new VoidPointable();
+ eval3 = args[3].createScalarEvaluator(ctx);
+ inputArg4 = new VoidPointable();
+ eval4 = args[4].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ byte[] data0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+
+ eval1.evaluate(tuple, inputArg1);
+ byte[] data1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+
+ eval2.evaluate(tuple, inputArg2);
+ byte[] data2 = inputArg2.getByteArray();
+ int offset2 = inputArg2.getStartOffset();
+
+ eval3.evaluate(tuple, inputArg3);
+ byte[] data3 = inputArg3.getByteArray();
+ int offset3 = inputArg3.getStartOffset();
+
+ eval4.evaluate(tuple, inputArg4);
+ byte[] data4 = inputArg4.getByteArray();
+ int offset4 = inputArg4.getStartOffset();
+
+ try {
+
+ OGCGeometry ogcGeometry =
+ OGCGeometry
+ .createFromEsriGeometry(
+ new Envelope(getVal(data0, offset0), getVal(data1, offset1),
+ getVal(data2, offset2), getVal(data3, offset3)),
+ SpatialReference.create((int) getVal(data4, offset4)));
+ ByteBuffer buffer = ogcGeometry.asBinary();
+ byte[] bytes = buffer.array();
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ out.writeInt(bytes.length);
+ out.write(bytes);
+ result.set(resultStorage);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
new file mode 100644
index 0000000..3c0ac97
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DDescriptor.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DDescriptor extends AbstractGetValDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMakePoint3DDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new STMakePoint3DEvaluator(args, ctx);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MAKE_POINT3D;
+ }
+
+ private class STMakePoint3DEvaluator implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+ private IPointable inputArg0;
+ private IPointable inputArg1;
+ private IPointable inputArg2;
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+ private final IScalarEvaluator eval2;
+ private Point point;
+ private AGeometry pointGeometry;
+
+ public STMakePoint3DEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg0 = new VoidPointable();
+ inputArg1 = new VoidPointable();
+ inputArg2 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ eval1 = args[1].createScalarEvaluator(ctx);
+ eval2 = args[2].createScalarEvaluator(ctx);
+ point = new Point(0, 0, 0);
+ pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+ eval2.evaluate(tuple, inputArg2);
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ byte[] bytes1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+ byte[] bytes2 = inputArg2.getByteArray();
+ int offset2 = inputArg2.getStartOffset();
+
+ resultStorage.reset();
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ point.setX(getVal(bytes0, offset0));
+ point.setY(getVal(bytes1, offset1));
+ point.setZ(getVal(bytes2, offset2));
+ AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+ } catch (IOException e1) {
+ throw HyracksDataException.create(e1);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
new file mode 100644
index 0000000..424fb8f
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePoint3DWithMDescriptor.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePoint3DWithMDescriptor extends AbstractGetValDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMakePoint3DWithMDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new STMakePoint3DWithMEvaluator(args, ctx);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MAKE_POINT3D_M;
+ }
+
+ private class STMakePoint3DWithMEvaluator implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+ private IPointable inputArg0;
+ private IPointable inputArg1;
+ private IPointable inputArg2;
+ private IPointable inputArg3;
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+ private final IScalarEvaluator eval2;
+ private final IScalarEvaluator eval3;
+ private Point point;
+ private AGeometry pointGeometry;
+
+ public STMakePoint3DWithMEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg0 = new VoidPointable();
+ inputArg1 = new VoidPointable();
+ inputArg2 = new VoidPointable();
+ inputArg3 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ eval1 = args[1].createScalarEvaluator(ctx);
+ eval2 = args[2].createScalarEvaluator(ctx);
+ eval3 = args[3].createScalarEvaluator(ctx);
+ point = new Point(0, 0, 0);
+ pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+ eval2.evaluate(tuple, inputArg2);
+ eval3.evaluate(tuple, inputArg3);
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ byte[] bytes1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+ byte[] bytes2 = inputArg2.getByteArray();
+ int offset2 = inputArg2.getStartOffset();
+ byte[] bytes3 = inputArg3.getByteArray();
+ int offset3 = inputArg3.getStartOffset();
+
+ resultStorage.reset();
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ point.setX(getVal(bytes0, offset0));
+ point.setY(getVal(bytes1, offset1));
+ point.setZ(getVal(bytes2, offset2));
+ point.setM(getVal(bytes3, offset3));
+ AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+ } catch (IOException e1) {
+ throw HyracksDataException.create(e1);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
new file mode 100644
index 0000000..2d901a7
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakePointDescriptor.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.om.base.AGeometry;
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STMakePointDescriptor extends AbstractGetValDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STMakePointDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new STMakePointEvaluator(args, ctx);
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_MAKE_POINT;
+ }
+
+ private class STMakePointEvaluator implements IScalarEvaluator {
+
+ private final ArrayBackedValueStorage resultStorage;
+ private final DataOutput out;
+ private IPointable inputArg0;
+ private IPointable inputArg1;
+ private final IScalarEvaluator eval0;
+ private final IScalarEvaluator eval1;
+ private Point point;
+ private AGeometry pointGeometry;
+
+ public STMakePointEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg0 = new VoidPointable();
+ inputArg1 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ eval1 = args[1].createScalarEvaluator(ctx);
+ point = new Point(0, 0);
+ pointGeometry = new AGeometry(new OGCPoint(point, SpatialReference.create(4326)));
+ }
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval0.evaluate(tuple, inputArg0);
+ eval1.evaluate(tuple, inputArg1);
+
+ byte[] bytes0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ byte[] bytes1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+
+ resultStorage.reset();
+ try {
+ out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ point.setX(getVal(bytes0, offset0));
+ point.setY(getVal(bytes1, offset1));
+ AGeometrySerializerDeserializer.INSTANCE.serialize(pointGeometry, out);
+ } catch (IOException e1) {
+ throw HyracksDataException.create(e1);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
new file mode 100644
index 0000000..8b1111b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNPointsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.GeometryCursor;
+import com.esri.core.geometry.MultiVertexGeometry;
+import com.esri.core.geometry.Point;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNPointsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STNPointsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ Geometry esriGeometry = geometry.getEsriGeometry();
+ if (esriGeometry != null && esriGeometry instanceof MultiVertexGeometry) {
+ return ((MultiVertexGeometry) esriGeometry).getPointCount();
+ } else if (esriGeometry instanceof Point) {
+ return 1;
+ } else if (esriGeometry == null) {
+ int count = 0;
+ GeometryCursor geometryCursor = geometry.getEsriGeometryCursor();
+ esriGeometry = geometryCursor.next();
+ while (esriGeometry != null) {
+ if (esriGeometry instanceof MultiVertexGeometry) {
+ count += ((MultiVertexGeometry) esriGeometry).getPointCount();
+ } else if (esriGeometry instanceof Point) {
+ count += 1;
+ }
+ esriGeometry = geometryCursor.next();
+ }
+ return count;
+ } else if (geometry.isEmpty()) {
+ return 0;
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_N_POINTS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.java
new file mode 100644
index 0000000..27e0490
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNRingsDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCMultiPolygon;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STNRingsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPolygon) {
+ return ((OGCPolygon) geometry).numInteriorRing() + 1;
+ } else if (geometry instanceof OGCMultiPolygon) {
+ OGCMultiPolygon polygon = (OGCMultiPolygon) geometry;
+ int numGeometries = polygon.numGeometries();
+ int count = 0;
+ for (int i = 1; i < numGeometries + 1; i++) {
+ if (polygon.geometryN(i) instanceof OGCPolygon) {
+ count += ((OGCPolygon) polygon.geometryN(i)).numInteriorRing() + 1;
+ }
+ }
+ return count;
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_N_RINGS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.java
new file mode 100644
index 0000000..b8d0c52
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumGeometriesDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumGeometriesDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STNumGeometriesDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCGeometryCollection) {
+ return ((OGCGeometryCollection) geometry).numGeometries();
+ } else if (!geometry.isEmpty()) {
+ return 1;
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_NUM_GEOMETRIIES;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.java
new file mode 100644
index 0000000..0c32ec5
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STNumInteriorRingsDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPolygon;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STNumInteriorRingsDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STNumInteriorRingsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPolygon) {
+ return ((OGCPolygon) geometry).numInteriorRing();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_NUM_INTERIOR_RINGS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.java
new file mode 100644
index 0000000..f131499
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STOverlapsDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STOverlapsDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STOverlapsDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.overlaps(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_OVERLAPS;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.java
new file mode 100644
index 0000000..e3ce3e0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPointNDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCLineString;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STPointNDescriptor extends AbstractSTGeometryNDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STPointNDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_POINT_N;
+ }
+
+ @Override
+ protected OGCGeometry evaluateOGCGeometry(OGCGeometry geometry, int n) throws HyracksDataException {
+ if (geometry instanceof OGCLineString) {
+ return ((OGCLineString) geometry).pointN(n);
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
new file mode 100644
index 0000000..56c79aa
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STPolygonizeDescriptor.java
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.ogc.OGCConcreteGeometryCollection;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCGeometryCollection;
+import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
+import org.apache.asterix.om.base.IAObject;
+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.AOrderedListType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class STPolygonizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STPolygonizeDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_POLYGONIZE;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+
+ return new STPolygonizeEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class STPolygonizeEvaluator implements IScalarEvaluator {
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+
+ public STPolygonizeEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx)
+ throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[0].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] bytes = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ AOrderedListType type = new AOrderedListType(BuiltinType.AGEOMETRY, null);
+ byte typeTag = inputArg.getByteArray()[inputArg.getStartOffset()];
+ ISerializerDeserializer serde;
+ if (typeTag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG) {
+ serde = new AOrderedListSerializerDeserializer(type);
+ } else if (typeTag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ serde = new AOrderedListSerializerDeserializer(type);
+ } else {
+ throw new TypeMismatchException(BuiltinFunctions.ST_POLYGONIZE, 0, typeTag,
+ ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG, ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG);
+ }
+
+ ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ IACursor cursor = ((IACollection) serde.deserialize(dataIn)).getCursor();
+ List<OGCGeometry> list = new ArrayList<>();
+ while (cursor.next()) {
+ IAObject object = cursor.get();
+ list.add(((AGeometry) object).getGeometry());
+ }
+ OGCGeometryCollection geometryCollection =
+ new OGCConcreteGeometryCollection(list, SpatialReference.create(4326));
+ try {
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AGEOMETRY)
+ .serialize(new AGeometry(geometryCollection), out);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
new file mode 100644
index 0000000..dc89943
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STRelateDescriptor.java
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ABoolean;
+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.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class STRelateDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STRelateDescriptor();
+ }
+ };
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_RELATE;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new STRelateEvaluator(args, ctx);
+ }
+ };
+ }
+
+ private class STRelateEvaluator implements IScalarEvaluator {
+ private ArrayBackedValueStorage resultStorage;
+ private DataOutput out;
+ private IPointable inputArg;
+ private IScalarEvaluator eval;
+ private IPointable inputArg0;
+ private IScalarEvaluator eval0;
+ private final IPointable inputArg1;
+ private final IScalarEvaluator eval1;
+
+ public STRelateEvaluator(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx) throws HyracksDataException {
+ resultStorage = new ArrayBackedValueStorage();
+ out = resultStorage.getDataOutput();
+ inputArg = new VoidPointable();
+ eval = args[2].createScalarEvaluator(ctx);
+ inputArg0 = new VoidPointable();
+ eval0 = args[0].createScalarEvaluator(ctx);
+ inputArg1 = new VoidPointable();
+ eval1 = args[1].createScalarEvaluator(ctx);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ eval.evaluate(tuple, inputArg);
+ byte[] bytes = inputArg.getByteArray();
+ int offset = inputArg.getStartOffset();
+ int len = inputArg.getLength();
+
+ eval0.evaluate(tuple, inputArg0);
+ byte[] bytes0 = inputArg0.getByteArray();
+ int offset0 = inputArg0.getStartOffset();
+ int len0 = inputArg0.getLength();
+
+ eval1.evaluate(tuple, inputArg1);
+ byte[] bytes1 = inputArg1.getByteArray();
+ int offset1 = inputArg1.getStartOffset();
+ int len1 = inputArg1.getLength();
+
+ if (bytes[offset] != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes[offset], ATypeTag.SERIALIZED_STRING_TYPE_TAG);
+ }
+ ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+ if (tag != ATypeTag.GEOMETRY) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes0[offset0],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+ if (tag != ATypeTag.GEOMETRY) {
+ throw new TypeMismatchException(getIdentifier(), 0, bytes1[offset1],
+ ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+
+ ByteArrayInputStream inStream = new ByteArrayInputStream(bytes, offset + 1, len - 1);
+ DataInputStream dataIn = new DataInputStream(inStream);
+ String matrix = AStringSerializerDeserializer.INSTANCE.deserialize(dataIn).getStringValue();
+ DataInputStream dataIn0 = new DataInputStream(new ByteArrayInputStream(bytes0, offset0 + 1, len0 - 1));
+ OGCGeometry geometry0 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn0).getGeometry();
+ DataInputStream dataIn1 = new DataInputStream(new ByteArrayInputStream(bytes1, offset1 + 1, len1 - 1));
+ OGCGeometry geometry1 = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn1).getGeometry();
+ try {
+ boolean val = geometry0.relate(geometry1, matrix);
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN)
+ .serialize(val ? ABoolean.TRUE : ABoolean.FALSE, out);
+ } catch (IOException e) {
+ throw new InvalidDataFormatException(getIdentifier(), e, ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+ }
+ result.set(resultStorage);
+ }
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.java
new file mode 100644
index 0000000..a46749d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSRIDDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSRIDDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STSRIDDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.SRID();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_SRID;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.java
new file mode 100644
index 0000000..988e781
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STStartPointDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCCurve;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STStartPointDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STStartPointDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCCurve) {
+ return ((OGCCurve) geometry).startPoint();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_START_POINT;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
new file mode 100644
index 0000000..e92f56b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STSymDifferenceDescriptor.java
@@ -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.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STSymDifferenceDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STSymDifferenceDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.symDifference(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_SYM_DIFFERENCE;
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.java
new file mode 100644
index 0000000..93a947a
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STTouchesDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STTouchesDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STTouchesDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.touches(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_TOUCHES;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
new file mode 100644
index 0000000..43891f0
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STUnionDescriptor.java
@@ -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.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STUnionDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STUnionDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.union(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_UNION;
+ }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.java
new file mode 100644
index 0000000..f563d61
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STWithinDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STWithinDescriptor extends AbstractSTDoubleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STWithinDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry0, OGCGeometry geometry1) throws HyracksDataException {
+ return geometry0.within(geometry1);
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_WITHIN;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.java
new file mode 100644
index 0000000..c796b1b
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STXDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPoint) {
+ return ((OGCPoint) geometry).X();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_X;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.java
new file mode 100644
index 0000000..19864bb
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMaxDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STXMaxDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ Geometry esriGeom = geometry.getEsriGeometry();
+ if (esriGeom != null) {
+ Envelope env = new Envelope();
+ esriGeom.queryEnvelope(env);
+ return env.getXMax();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_X_MAX;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.java
new file mode 100644
index 0000000..12378c6
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STXMinDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STXMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STXMinDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ Geometry esriGeom = geometry.getEsriGeometry();
+ if (esriGeom != null) {
+ Envelope env = new Envelope();
+ esriGeom.queryEnvelope(env);
+ return env.getXMin();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_X_MIN;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.java
new file mode 100644
index 0000000..782370d
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STYDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPoint) {
+ return ((OGCPoint) geometry).Y();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Y;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.java
new file mode 100644
index 0000000..83ea647
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMaxDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STYMaxDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ Geometry esriGeom = geometry.getEsriGeometry();
+ if (esriGeom != null) {
+ Envelope env = new Envelope();
+ esriGeom.queryEnvelope(env);
+ return env.getYMax();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Y_MAX;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.java
new file mode 100644
index 0000000..c6b5f36
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STYMinDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.Geometry;
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STYMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STYMinDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ Geometry esriGeom = geometry.getEsriGeometry();
+ if (esriGeom != null) {
+ Envelope env = new Envelope();
+ esriGeom.queryEnvelope(env);
+ return env.getYMin();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Y_MIN;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.java
new file mode 100644
index 0000000..b5b6d78
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZDescriptor.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.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.esri.core.geometry.ogc.OGCPoint;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STZDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ if (geometry instanceof OGCPoint) {
+ return ((OGCPoint) geometry).Z();
+ } else {
+ throw new UnsupportedOperationException(
+ "The operation " + getIdentifier() + " is not supported for the type " + geometry.geometryType());
+ }
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Z;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.java
new file mode 100644
index 0000000..7933f08
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMaxDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMaxDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STZMaxDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.MaxZ();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Z_MAX;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.java
new file mode 100644
index 0000000..d0da92e
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STZMinDescriptor.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.asterix.geo.evaluators.functions;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class STZMinDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new STZMinDescriptor();
+ }
+ };
+
+ @Override
+ protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+ return geometry.MinZ();
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.ST_Z_MIN;
+ }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
new file mode 100644
index 0000000..d9529e2
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/resources/META-INF/services/org.apache.asterix.om.functions.IFunctionRegistrant
@@ -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.
+#
+
+org.apache.asterix.geo.evaluators.GeoFunctionRegistrant
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 2d87556..856073d 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -147,6 +147,7 @@
import org.apache.asterix.lang.common.statement.UpsertStatement;
import org.apache.asterix.lang.common.statement.WriteStatement;
import org.apache.asterix.lang.common.struct.Identifier;
+import org.apache.asterix.lang.common.struct.OperatorType;
import org.apache.asterix.lang.common.struct.QuantifiedPair;
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -1811,23 +1812,25 @@
Expression MultExpr()throws ParseException:
{
OperatorExpr op = null;
+ OperatorType opType = null;
Expression operand = null;
}
{
operand = ExponentExpr()
- (( <MUL> | <DIV> | <MOD> | <IDIV>)
+ ( (
+ <MUL> { opType = OperatorType.MUL; } |
+ <DIVIDE> { opType = OperatorType.DIVIDE; } |
+ <DIV> { opType = OperatorType.DIV; } |
+ ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+ )
{
if (op == null) {
op = new OperatorExpr();
op.addOperand(operand);
op.setCurrentop(true);
}
- try{
- op.addOperator(token.image);
- } catch (CompilationException e){
- throw new ParseException(e.getMessage());
- }
+ op.addOperator(opType);
}
operand = ExponentExpr()
{
@@ -2724,10 +2727,10 @@
TOKEN :
{
<CARET : "^">
- | <DIV : "/">
- | <IDIV : "idiv">
+ | <DIVIDE : "/">
+ | <DIV : "div">
| <MINUS : "-">
- | <MOD : "%">
+ | <MOD : "mod">
| <MUL : "*">
| <PLUS : "+">
@@ -2739,6 +2742,7 @@
| <COLON : ":">
| <COMMA : ",">
| <DOT : ".">
+ | <PERCENT: "%">
| <QUES : "?">
| <LT : "<">
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
index 90b457e..9521469 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/OperatorExpr.java
@@ -84,10 +84,18 @@
exprList.add(operand);
}
- public static final boolean opIsComparison(OperatorType t) {
- boolean cmp = t == OperatorType.EQ || t == OperatorType.NEQ || t == OperatorType.GT;
- cmp = cmp || t == OperatorType.GE || t == OperatorType.LT || t == OperatorType.LE;
- return cmp;
+ public static boolean opIsComparison(OperatorType t) {
+ switch (t) {
+ case EQ:
+ case NEQ:
+ case GT:
+ case GE:
+ case LT:
+ case LE:
+ return true;
+ default:
+ return false;
+ }
}
public void addOperator(String strOp) throws CompilationException {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
index e11f169..f4f2ae1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/struct/OperatorType.java
@@ -31,11 +31,10 @@
MINUS("-"),
CONCAT("||"),
MUL("*"),
- DIV("/"), // float/double
- // divide
- MOD("%"),
+ DIVIDE("/"),
+ DIV("div"),
+ MOD("mod"),
CARET("^"),
- IDIV("idiv"), // integer divide
FUZZY_EQ("~="),
LIKE("like"),
NOT_LIKE("not_like"),
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
index 3dffc00..83dc2f1b 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/CommonFunctionMapUtil.java
@@ -39,7 +39,6 @@
addFunctionMapping("upper", "uppercase"); // upper, internal: uppercase
addFunctionMapping("title", "initcap"); // title, internal: initcap
addFunctionMapping("regexp_contains", "matches"); // regexp_contains, internal: matches
- addFunctionMapping("power", "caret"); //pow, internal: caret
addFunctionMapping("int", "integer"); // int, internal: integer
// The "mapped-to" names are to be deprecated.
@@ -69,6 +68,11 @@
addFunctionMapping("ifinf", "if-inf"); // ifinf, internal: if-inf
addFunctionMapping("ifnan", "if-nan"); // ifnan, internal: if-nan
addFunctionMapping("ifnanorinf", "if-nan-or-inf"); // ifnanorinf, internal: if-nan-or-inf
+ addFunctionMapping("missingif", "missing-if"); // missingif, internal: missing-if
+ addFunctionMapping("nanif", "nan-if"); // nanif, internal: nan-if
+ addFunctionMapping("neginfif", "neginf-if"); // neginfif, internal: neginf-if
+ addFunctionMapping("nullif", "null-if"); // nullif, internal: null-if
+ addFunctionMapping("posinfif", "posinf-if"); // posinfif, internal: posinf-if
addFunctionMapping("toarray", "to-array"); // toarray, internal: to-array
addFunctionMapping("toatomic", "to-atomic"); // toatomic, internal: to-atomic
addFunctionMapping("toatom", "to-atomic"); // toatom, internal: to-atomic
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index b8b7622..c3a9c0b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -1790,7 +1790,6 @@
(
LOOKAHEAD(2)
expr = OperatorExpr()
- | expr = CaseExpr()
| expr = QuantifiedExpression()
)
{
@@ -2019,7 +2018,7 @@
<NULL> { fn = BuiltinFunctions.IS_NULL; } |
<MISSING> { fn = BuiltinFunctions.IS_MISSING; } |
<UNKNOWN> { fn = BuiltinFunctions.IS_UNKNOWN; } |
- <VALUED> { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
+ (<KNOWN> | <VALUED>) { not = !not; fn = BuiltinFunctions.IS_UNKNOWN; }
)
{
FunctionSignature signature = new FunctionSignature(fn);
@@ -2144,23 +2143,25 @@
Expression MultExpr()throws ParseException:
{
OperatorExpr op = null;
+ OperatorType opType = null;
Expression operand = null;
}
{
operand = ExponentExpr()
- (( <MUL> | <DIV> | <MOD> | <IDIV>)
+ ( (
+ <MUL> { opType = OperatorType.MUL; } |
+ <DIVIDE> { opType = OperatorType.DIVIDE; } |
+ <DIV> { opType = OperatorType.DIV; } |
+ ( <MOD> | <PERCENT> ) { opType = OperatorType.MOD; }
+ )
{
if (op == null) {
op = new OperatorExpr();
op.addOperand(operand);
op.setCurrentop(true);
}
- try{
- op.addOperator(token.image);
- } catch (Exception e){
- throw new ParseException(e.getMessage());
- }
+ op.addOperator(opType);
}
operand = ExponentExpr()
{
@@ -2306,6 +2307,7 @@
{
( LOOKAHEAD(4)
expr = FunctionCallExpr()
+ | expr = CaseExpr()
| expr = Literal()
| expr = VariableRef()
| expr = ListConstructor()
@@ -3253,6 +3255,7 @@
| <JOIN : "join">
| <KEYWORD : "keyword">
| <KEY : "key">
+ | <KNOWN : "known">
| <LEFT : "left">
| <LETTING : "letting">
| <LET : "let">
@@ -3310,10 +3313,10 @@
{
<CARET : "^">
| <CONCAT : "||">
- | <DIV : "/">
- | <IDIV : "idiv">
+ | <DIVIDE : "/">
+ | <DIV : "div">
| <MINUS : "-">
- | <MOD : "%">
+ | <MOD : "mod">
| <MUL : "*">
| <PLUS : "+">
@@ -3326,6 +3329,7 @@
| <COLON : ":">
| <COMMA : ",">
| <DOT : ".">
+ | <PERCENT: "%">
| <QUES : "?">
| <SEMICOLON : ";">
| <SHARP : "#">
diff --git a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 3613166..044852b 100644
--- a/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterixdb/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -124,6 +124,62 @@
}
}
+ public void getLastTokenImage(TokenImage image) {
+ if (bufpos >= tokenBegin) {
+ image.reset(buffer, tokenBegin, bufpos - tokenBegin);
+ } else {
+ image.reset(buffer, tokenBegin, bufsize - tokenBegin, buffer, 0, bufpos);
+ }
+ }
+
+ public static class TokenImage{
+ private char[] buffer;
+ private int begin;
+ private int length;
+
+ // used to hold circular tokens
+ private char[] tmpBuffer;
+
+ private static final double TMP_BUFFER_GROWTH = 1.5;
+
+ public void reset(char[] buffer, int begin, int length) {
+ this.buffer = buffer;
+ this.begin = begin;
+ this.length = length;
+ }
+
+ public void reset(char[] buffer, int begin, int length, char[] extraBuffer, int extraBegin, int extraLength) {
+ ensureTmpBufferSize(length + extraLength);
+ System.arraycopy(buffer, begin, tmpBuffer, 0, length);
+ System.arraycopy(extraBuffer, extraBegin, tmpBuffer, length, extraLength);
+ this.buffer = tmpBuffer;
+ this.begin = 0;
+ this.length = length + extraLength;
+ }
+
+ public char[] getBuffer() {
+ return buffer;
+ }
+
+ public int getBegin() {
+ return begin;
+ }
+
+ public int getLength() {
+ return length;
+ }
+
+
+ private void ensureTmpBufferSize(int size) {
+ int oldSize = tmpBuffer!=null?tmpBuffer.length:0;
+ if(oldSize < size) {
+ int newSize = Math.max((int)(oldSize * TMP_BUFFER_GROWTH), size);
+ tmpBuffer = new char[newSize];
+ }
+ }
+
+ }
+
public int getColumn() {
return column;
}
@@ -262,4 +318,4 @@
endOf_UNUSED_Buffer = bufsize;
tokenBegin = 0;
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
index be295d6..1cfb804 100644
--- a/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
+++ b/asterixdb/asterix-maven-plugins/record-manager-generator-maven-plugin/src/main/resources/ArenaManager.java
@@ -56,7 +56,7 @@
public long allocate() {
final LocalManager localManager = local.get();
final @E@RecordManager recMgr = localManager.mgr;
- final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7fff) : 0;
+ final int allocId = TRACK_ALLOC_ID ? (++recMgr.allocCounter % 0x7ffe + 1) : 1;
final int localId = recMgr.allocate();
long result = TypeUtil.Global.build(localManager.arenaId, allocId, localId);
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 a6a708f..4474684 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
@@ -97,7 +97,6 @@
private final MetadataCache cache = new MetadataCache();
protected final Collection<IAsterixStateProxy> proxies;
protected IMetadataNode metadataNode;
- private final ReadWriteLock metadataLatch;
protected boolean rebindMetadataNode = false;
// TODO(mblow): replace references of this (non-constant) field with a method,
@@ -117,7 +116,6 @@
throw new IllegalArgumentException("Null / empty list of proxies given to MetadataManager");
}
this.proxies = proxies;
- this.metadataLatch = new ReentrantReadWriteLock(true);
}
protected abstract TxnId createTxnId();
@@ -725,26 +723,6 @@
}
@Override
- public void acquireWriteLatch() {
- metadataLatch.writeLock().lock();
- }
-
- @Override
- public void releaseWriteLatch() {
- metadataLatch.writeLock().unlock();
- }
-
- @Override
- public void acquireReadLatch() {
- metadataLatch.readLock().lock();
- }
-
- @Override
- public void releaseReadLatch() {
- metadataLatch.readLock().unlock();
- }
-
- @Override
public FeedPolicyEntity getFeedPolicy(MetadataTransactionContext ctx, String dataverse, String policyName)
throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
index 7103fd8..baface2 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IActiveEntityController.java
@@ -82,7 +82,7 @@
void add(Dataset dataset) throws HyracksDataException;
/**
- * Remove dataset to the list of associated datasets
+ * Remove dataset from the list of associated datasets
*
* @param dataset
* the dataset to add
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 966792e..a646893 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
@@ -519,14 +519,6 @@
int getMostRecentDatasetId() throws AlgebricksException;
- void acquireWriteLatch();
-
- void releaseWriteLatch();
-
- void acquireReadLatch();
-
- void releaseReadLatch();
-
/**
* Removes a library , acquiring local locks on behalf of the given transaction
* id.
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 980d36d..95526f4 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
@@ -24,17 +24,19 @@
import java.util.HashMap;
import java.util.List;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.config.MetadataProperties;
import org.apache.asterix.common.context.AsterixVirtualBufferCacheProvider;
import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
import org.apache.asterix.external.api.IAdapterFactory;
@@ -66,7 +68,6 @@
import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerFactory;
import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerFactory;
import org.apache.asterix.transaction.management.resource.DatasetLocalResourceFactory;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -78,7 +79,6 @@
import org.apache.hyracks.storage.am.common.build.IndexBuilder;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelper;
import org.apache.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeLocalResourceFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -324,10 +324,10 @@
ILSMOperationTrackerFactory opTrackerFactory =
index.isPrimaryIndex() ? new PrimaryIndexOperationTrackerFactory(datasetId)
: new SecondaryIndexOperationTrackerFactory(datasetId);
- ILSMComponentIdGeneratorFactory idGeneratorProvider =
- new DatasetLSMComponentIdGeneratorFactory(index.getDatasetId().getId());
+ ILSMComponentIdGeneratorFactory idGeneratorProvider = new DatasetLSMComponentIdGeneratorFactory(datasetId);
+ DatasetInfoProvider datasetInfoProvider = new DatasetInfoProvider(datasetId);
ILSMIOOperationCallbackFactory ioOpCallbackFactory =
- new LSMBTreeIOOperationCallbackFactory(idGeneratorProvider);
+ new LSMIndexIOOperationCallbackFactory(idGeneratorProvider, datasetInfoProvider);
IStorageComponentProvider storageComponentProvider = appContext.getStorageComponentProvider();
if (isNewUniverse()) {
final double bloomFilterFalsePositiveRate =
@@ -375,35 +375,23 @@
// as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then
// to INDEX_DATASET.
MetadataTransactionContext mdTxnCtx = null;
- MetadataManager.INSTANCE.acquireWriteLatch();
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Starting DDL recovery ...");
- }
-
+ LOGGER.info("Starting DDL recovery ...");
try {
mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
for (Dataverse dataverse : dataverses) {
recoverDataverse(mdTxnCtx, dataverse);
}
- // the commit wasn't there before. yet, everything was working
- // correctly!!!!!!!!!!!
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Completed DDL recovery.");
- }
+ LOGGER.info("Completed DDL recovery.");
} catch (Exception e) {
try {
- if (IS_DEBUG_MODE) {
- LOGGER.log(Level.ERROR, "Failure during DDL recovery", e);
- }
+ LOGGER.error("Failure during DDL recovery", e);
MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
} catch (Exception e2) {
e.addSuppressed(e2);
}
- throw new MetadataException(e);
- } finally {
- MetadataManager.INSTANCE.releaseWriteLatch();
+ throw MetadataException.create(e);
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
index 66283d6..88449f9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DataSource.java
@@ -40,6 +40,7 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public abstract class DataSource implements IDataSource<DataSourceId> {
@@ -159,7 +160,8 @@
public abstract Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException;
}
\ No newline at end of file
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
index ddab5bc..62cce05 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/DatasetDataSource.java
@@ -21,6 +21,8 @@
import java.util.List;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataManager;
@@ -42,10 +44,11 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public class DatasetDataSource extends DataSource {
- private Dataset dataset;
+ private final Dataset dataset;
public DatasetDataSource(DataSourceId id, Dataset dataset, IAType itemType, IAType metaItemType,
byte datasourceType, IDatasetDetails datasetDetails, INodeDomain datasetDomain) throws AlgebricksException {
@@ -92,11 +95,16 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException {
switch (dataset.getDatasetType()) {
case EXTERNAL:
+ if (tupleFilterFactory != null || outputLimit >= 0) {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Tuple filter and limit are not supported by ExternalDataSource");
+ }
Dataset externalDataset = ((DatasetDataSource) dataSource).getDataset();
String itemTypeName = externalDataset.getItemTypeName();
IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
@@ -117,7 +125,8 @@
int[] maxFilterFieldIndexes = createFilterIndexes(maxFilterVars, opSchema);
return metadataProvider.buildBtreeRuntime(jobSpec, opSchema, typeEnv, context, true, false,
((DatasetDataSource) dataSource).getDataset(), primaryIndex.getIndexName(), null, null, true,
- true, false, minFilterFieldIndexes, maxFilterFieldIndexes, false);
+ true, false, minFilterFieldIndexes, maxFilterFieldIndexes, tupleFilterFactory, outputLimit,
+ false);
default:
throw new AlgebricksException("Unknown datasource type");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
index 5c3ed56..81be1f7 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FeedDataSource.java
@@ -22,7 +22,8 @@
import java.util.List;
import org.apache.asterix.active.EntityId;
-import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.operators.FeedCollectOperatorDescriptor;
import org.apache.asterix.external.util.FeedUtils.FeedRuntimeType;
@@ -48,6 +49,7 @@
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public class FeedDataSource extends DataSource implements IMutationDataSource {
@@ -158,10 +160,15 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException {
try {
+ if (tupleFilterFactory != null || outputLimit >= 0) {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Tuple filter and limit are not supported by FeedDataSource");
+ }
ARecordType feedOutputType = (ARecordType) itemType;
ISerializerDeserializer payloadSerde =
metadataProvider.getDataFormat().getSerdeProvider().getSerializerDeserializer(feedOutputType);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
index d2b9871..3b5cf2e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSource.java
@@ -25,6 +25,8 @@
import java.util.Set;
import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
import org.apache.asterix.metadata.api.IDatasourceFunction;
import org.apache.asterix.om.types.IAType;
@@ -44,6 +46,7 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public abstract class FunctionDataSource extends DataSource {
@@ -68,9 +71,14 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException {
+ if (tupleFilterFactory != null || outputLimit >= 0) {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "tuple filter and limit are not supported by FunctionDataSource");
+ }
GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
adapterFactory.setOutputType(RecordUtil.FULLY_OPEN_RECORD_TYPE);
IClusterStateManager csm = metadataProvider.getApplicationContext().getClusterStateManager();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 8fdcbbc..3460a46 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -24,6 +24,8 @@
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.external.api.IAdapterFactory;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
@@ -44,6 +46,7 @@
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.ITupleFilterFactory;
public class LoadableDataSource extends DataSource {
@@ -124,9 +127,14 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildDatasourceScanRuntime(
MetadataProvider metadataProvider, IDataSource<DataSourceId> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException {
+ if (tupleFilterFactory != null || outputLimit >= 0) {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "tuple filter and limit are not supported by LoadableDataSource");
+ }
LoadableDataSource alds = (LoadableDataSource) dataSource;
ARecordType itemType = (ARecordType) alds.getLoadedType();
IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
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 a081fb4..7f8d31d 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
@@ -26,7 +26,6 @@
import java.util.List;
import java.util.Map;
-import org.apache.asterix.common.transactions.ITxnIdFactory;
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -38,6 +37,7 @@
import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.metadata.LockList;
+import org.apache.asterix.common.transactions.ITxnIdFactory;
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
@@ -132,6 +132,7 @@
import org.apache.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
@@ -380,10 +381,12 @@
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
IDataSource<DataSourceId> dataSource, List<LogicalVariable> scanVariables,
List<LogicalVariable> projectVariables, boolean projectPushed, List<LogicalVariable> minFilterVars,
- List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
- JobGenContext context, JobSpecification jobSpec, Object implConfig) throws AlgebricksException {
+ List<LogicalVariable> maxFilterVars, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec,
+ Object implConfig) throws AlgebricksException {
return ((DataSource) dataSource).buildDatasourceScanRuntime(this, dataSource, scanVariables, projectVariables,
- projectPushed, minFilterVars, maxFilterVars, opSchema, typeEnv, context, jobSpec, implConfig);
+ projectPushed, minFilterVars, maxFilterVars, tupleFilterFactory, outputLimit, opSchema, typeEnv,
+ context, jobSpec, implConfig);
}
protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
@@ -433,7 +436,8 @@
IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, boolean retainInput,
boolean retainMissing, Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, boolean propagateFilter, int[] minFilterFieldIndexes,
- int[] maxFilterFieldIndexes, boolean isIndexOnlyPlan) throws AlgebricksException {
+ int[] maxFilterFieldIndexes, ITupleFilterFactory tupleFilterFactory, long outputLimit,
+ boolean isIndexOnlyPlan) throws AlgebricksException {
boolean isSecondary = true;
Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
dataset.getDatasetName(), dataset.getDatasetName());
@@ -477,8 +481,8 @@
btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
context.getMissingWriterFactory(), searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, propagateFilter, proceedIndexOnlyPlan, failValueForIndexOnlyPlan,
- successValueForIndexOnlyPlan);
+ maxFilterFieldIndexes, propagateFilter, tupleFilterFactory, outputLimit, proceedIndexOnlyPlan,
+ failValueForIndexOnlyPlan, successValueForIndexOnlyPlan);
} else {
btreeSearchOp = new ExternalBTreeSearchOperatorDescriptor(jobSpec, outputRecDesc, lowKeyFields,
highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory, retainInput, retainMissing,
@@ -1541,7 +1545,7 @@
}
}
- private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+ public AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
throws AlgebricksException {
// No filtering condition.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index a531add..14f76eb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -79,6 +79,7 @@
_builtinTypeMap.put("null", BuiltinType.ANULL);
_builtinTypeMap.put("uuid", BuiltinType.AUUID);
_builtinTypeMap.put("shortwithouttypeinfo", BuiltinType.SHORTWITHOUTTYPEINFO);
+ _builtinTypeMap.put("geometry", BuiltinType.AGEOMETRY);
}
private BuiltinTypeMap() {
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 7981309..2975972 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
@@ -27,17 +27,17 @@
import org.apache.asterix.active.IActiveEntityEventsListener;
import org.apache.asterix.active.IActiveNotificationHandler;
+import org.apache.asterix.common.api.IDatasetInfoProvider;
+import org.apache.asterix.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.context.CorrelatedPrefixMergePolicyFactory;
+import org.apache.asterix.common.context.DatasetInfoProvider;
import org.apache.asterix.common.context.DatasetLSMComponentIdGeneratorFactory;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.NoOpFrameOperationCallbackFactory;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMBTreeWithBuddyIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import org.apache.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
import org.apache.asterix.common.metadata.IDataset;
import org.apache.asterix.common.transactions.IRecoveryManager.ResourceType;
import org.apache.asterix.common.utils.JobUtils;
@@ -105,7 +105,6 @@
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.IFrameOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicyFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -502,24 +501,9 @@
* @throws AlgebricksException
* if the factory could not be created for the index/dataset combination
*/
+ @SuppressWarnings("squid:S1172")
public ILSMIOOperationCallbackFactory getIoOperationCallbackFactory(Index index) throws AlgebricksException {
- switch (index.getIndexType()) {
- case BTREE:
- return getDatasetType() == DatasetType.EXTERNAL
- && !index.getIndexName().equals(IndexingConstants.getFilesIndexName(getDatasetName()))
- ? new LSMBTreeWithBuddyIOOperationCallbackFactory(getComponentIdGeneratorFactory())
- : new LSMBTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
- case RTREE:
- return new LSMRTreeIOOperationCallbackFactory(getComponentIdGeneratorFactory());
- case LENGTH_PARTITIONED_NGRAM_INVIX:
- case LENGTH_PARTITIONED_WORD_INVIX:
- case SINGLE_PARTITION_NGRAM_INVIX:
- case SINGLE_PARTITION_WORD_INVIX:
- return new LSMInvertedIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory());
- default:
- throw new CompilationException(ErrorCode.COMPILATION_UNKNOWN_INDEX_TYPE,
- index.getIndexType().toString());
- }
+ return new LSMIndexIOOperationCallbackFactory(getComponentIdGeneratorFactory(), getDatasetInfoProvider());
}
/**
@@ -538,6 +522,10 @@
return new DatasetLSMComponentIdGeneratorFactory(getDatasetId());
}
+ public IDatasetInfoProvider getDatasetInfoProvider() {
+ return new DatasetInfoProvider(getDatasetId());
+ }
+
/**
* Get search callback factory for this dataset with the passed index and operation
*
@@ -563,7 +551,6 @@
boolean proceedIndexOnlyPlan) throws AlgebricksException {
if (index.isPrimaryIndex()) {
/**
- /*
* Due to the read-committed isolation level,
* we may acquire very short duration lock(i.e., instant lock) for readers.
*/
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 bf19b03..3159030 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
@@ -22,6 +22,7 @@
import java.util.List;
import org.apache.asterix.common.config.DatasetConfig.IndexType;
+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.transactions.IRecoveryManager.ResourceType;
@@ -153,8 +154,18 @@
ARecordType recType) throws AlgebricksException {
Pair<IAType, Boolean> keyPairType = null;
IAType subType = recType;
+ boolean nullable = false;
for (int i = 0; i < fieldName.size(); i++) {
- subType = ((ARecordType) subType).getFieldType(fieldName.get(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 {
+ throw AsterixException.create(ErrorCode.COMPILATION_ILLEGAL_STATE, "Unexpected type " + fieldType);
+ }
+
if (subType == null) {
keyPairType = Index.getNonNullableType(fieldType);
break;
@@ -163,13 +174,16 @@
if (subType != null) {
keyPairType = Index.getNonNullableKeyFieldType(fieldName, recType);
}
+ keyPairType.second = keyPairType.second || nullable;
return keyPairType;
}
public static Pair<IAType, Boolean> getNonNullableKeyFieldType(List<String> expr, ARecordType recType)
throws AlgebricksException {
IAType keyType = Index.keyFieldType(expr, recType);
- return getNonNullableType(keyType);
+ Pair<IAType, Boolean> pair = getNonNullableType(keyType);
+ pair.second = pair.second || recType.isSubFieldNullable(expr);
+ return pair;
}
private static IAType keyFieldType(List<String> expr, ARecordType recType) throws AlgebricksException {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 8a0cf84..3ae0fec 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -296,7 +296,6 @@
}
MetadataTransactionContext ctx = null;
- MetadataManager.INSTANCE.acquireReadLatch();
try {
ctx = MetadataManager.INSTANCE.beginTransaction();
Datatype t = MetadataManager.INSTANCE.getDatatype(ctx, dataverseName, datatypeName);
@@ -314,8 +313,6 @@
}
throw new MetadataException(ErrorCode.FEED_CREATE_FEED_DATATYPE_ERROR, e, datatypeName);
}
- } finally {
- MetadataManager.INSTANCE.releaseReadLatch();
}
return outputType;
}
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 45cf7c8..3be3e2f 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -122,5 +122,10 @@
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.esri.geometry</groupId>
+ <artifactId>esri-geometry-api</artifactId>
+ <version>2.0.0</version>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 95aef79..40b42eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -69,7 +69,7 @@
private int[] openFieldNameLengths;
private int numberOfOpenFields;
- private RuntimeRecordTypeInfo recTypeInfo;
+ private final RuntimeRecordTypeInfo recTypeInfo;
public RecordBuilder() {
this.closedPartOutputStream = new ByteArrayAccessibleOutputStream();
@@ -302,12 +302,7 @@
@Override
public int getFieldId(String fieldName) {
- for (int i = 0; i < recType.getFieldNames().length; i++) {
- if (recType.getFieldNames()[i].equals(fieldName)) {
- return i;
- }
- }
- return -1;
+ return recType.getFieldIndex(fieldName);
}
public IBinaryHashFunction getFieldNameHashFunction() {
@@ -317,4 +312,4 @@
public IBinaryComparator getFieldNameComparator() {
return utf8Comparator;
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
index 21880dd..e58f210 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
@@ -20,6 +20,7 @@
import java.io.PrintStream;
+import org.apache.asterix.dataflow.data.nontagged.printers.json.clean.AGeometryPrinterFactory;
import org.apache.asterix.om.pointables.AListVisitablePointable;
import org.apache.asterix.om.pointables.ARecordVisitablePointable;
import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
@@ -116,6 +117,9 @@
case SHORTWITHOUTTYPEINFO:
ShortWithoutTypeInfoPrinterFactory.PRINTER.print(b, s, l, ps);
return true;
+ case GEOMETRY:
+ AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
default:
return false;
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
new file mode 100644
index 0000000..8e51e35
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AGeometryPrinterFactory.java
@@ -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.
+ */
+package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.PrintStream;
+
+public class AGeometryPrinterFactory implements IPrinterFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final AGeometryPrinterFactory INSTANCE = new AGeometryPrinterFactory();
+
+ public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+ ByteArrayInputStream inStream = new ByteArrayInputStream(b, s + 1, l - 1);
+ DataInput dataIn = new DataInputStream(inStream);
+ OGCGeometry geometry = AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+ ps.print(geometry.asGeoJson());
+ };
+
+ @Override
+ public IPrinter createPrinter() {
+ return PRINTER;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
index 5cae68c..27fc7eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
@@ -113,6 +113,9 @@
case UUID:
AUUIDPrinterFactory.PRINTER.print(b, s, l, ps);
return true;
+ case GEOMETRY:
+ AGeometryPrinterFactory.PRINTER.print(b, s, l, ps);
+ return true;
default:
return false;
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
new file mode 100644
index 0000000..0a74ab7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AGeometrySerializerDeserializer.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.om.base.AGeometry;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.esri.core.geometry.OperatorImportFromWkb;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WkbImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AGeometrySerializerDeserializer implements ISerializerDeserializer<AGeometry> {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final AGeometrySerializerDeserializer INSTANCE = new AGeometrySerializerDeserializer();
+
+ /**Use WGS 84 (EPSG:4326) as the default coordinate reference system*/
+ public static final SpatialReference DEFAULT_CRS = SpatialReference.create(4326);
+
+ private AGeometrySerializerDeserializer() {
+ }
+
+ @Override
+ public AGeometry deserialize(DataInput in) throws HyracksDataException {
+ try {
+ int length = in.readInt();
+ byte[] bytes = new byte[length];
+ in.readFully(bytes);
+ ByteBuffer buffer = ByteBuffer.wrap(bytes);
+ OGCGeometry geometry = OGCGeometry.createFromOGCStructure(
+ OperatorImportFromWkb.local().executeOGC(WkbImportFlags.wkbImportDefaults, buffer, null),
+ DEFAULT_CRS);
+ return new AGeometry(geometry);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ @Override
+ public void serialize(AGeometry instance, DataOutput out) throws HyracksDataException {
+ try {
+ OGCGeometry geometry = instance.getGeometry();
+ byte[] buffer = geometry.asBinary().array();
+ // For efficiency, we store the size of the geometry in bytes in the first 32 bits
+ // This allows AsterixDB to skip over this attribute if needed.
+ out.writeInt(buffer.length);
+ out.write(buffer);
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index aef4ca1..57f3449 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -30,6 +30,7 @@
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.ADuration;
import org.apache.asterix.om.base.AFloat;
+import org.apache.asterix.om.base.AGeometry;
import org.apache.asterix.om.base.AInt16;
import org.apache.asterix.om.base.AInt32;
import org.apache.asterix.om.base.AInt64;
@@ -120,6 +121,8 @@
return AOrderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
case MULTISET:
return AUnorderedListSerializerDeserializer.SCHEMALESS_INSTANCE.deserialize(in);
+ case GEOMETRY:
+ return AGeometrySerializerDeserializer.INSTANCE.deserialize(in);
default:
throw new NotImplementedException("No serializer/deserializer implemented for type " + typeTag + " .");
}
@@ -213,6 +216,9 @@
case TYPE:
ATypeSerializerDeserializer.INSTANCE.serialize((IAType) instance, out);
break;
+ case GEOMETRY:
+ AGeometrySerializerDeserializer.INSTANCE.serialize((AGeometry) instance, out);
+ break;
default:
throw new HyracksDataException(
"No serializer/deserializer implemented for type " + t.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
index 888b34c..4bb9f08 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AStringSerializerDeserializer.java
@@ -69,4 +69,8 @@
throw HyracksDataException.create(e);
}
}
-}
+
+ public void serialize(char[] buffer, int start, int length, DataOutput out) throws IOException {
+ UTF8StringUtil.writeUTF8(buffer, start, length, out, utf8StringWriter);
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
index fd4df0d..6b33408 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/BinaryBooleanInspector.java
@@ -18,11 +18,14 @@
*/
package org.apache.asterix.formats.nontagged;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.EnumDeserializer;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
public class BinaryBooleanInspector implements IBinaryBooleanInspector {
private static final BinaryBooleanInspector INSTANCE = new BinaryBooleanInspector();
@@ -36,11 +39,13 @@
}
};
+ private static final String NAME = "boolean-inspector";
+
private BinaryBooleanInspector() {
}
@Override
- public boolean getBooleanValue(byte[] bytes, int offset, int length) {
+ public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException {
byte serializedTypeTag = bytes[offset];
if (serializedTypeTag == ATypeTag.SERIALIZED_MISSING_TYPE_TAG
|| serializedTypeTag == ATypeTag.SERIALIZED_NULL_TYPE_TAG) {
@@ -49,8 +54,9 @@
/** check if the runtime type is boolean */
ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serializedTypeTag);
if (typeTag != ATypeTag.BOOLEAN) {
- throw new IllegalStateException("Runtime error: the select condition should be of the boolean type!");
+ throw new RuntimeDataException(ErrorCode.TYPE_MISMATCH, NAME, 0, ATypeTag.BOOLEAN, typeTag);
}
+
return bytes[offset + 1] == 1;
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
index 81dc0c2..d66673d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/SerializerDeserializerProvider.java
@@ -32,6 +32,7 @@
import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
@@ -162,6 +163,8 @@
return AUUIDSerializerDeserializer.INSTANCE;
case SHORTWITHOUTTYPEINFO:
return ShortSerializerDeserializer.INSTANCE;
+ case GEOMETRY:
+ return AGeometrySerializerDeserializer.INSTANCE;
default:
throw new NotImplementedException(
"No serializer/deserializer implemented for type " + aqlType.getTypeTag() + " .");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.java
new file mode 100644
index 0000000..3b9c55d
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AGeometry.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.om.base;
+
+import com.esri.core.geometry.ogc.OGCGeometry;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+
+import java.io.IOException;
+
+public class AGeometry implements IAObject {
+
+ protected OGCGeometry geometry;
+
+ public AGeometry(OGCGeometry geometry) {
+ this.geometry = geometry;
+ }
+
+ public OGCGeometry getGeometry() {
+ return geometry;
+ }
+
+ @Override
+ public IAType getType() {
+ return BuiltinType.AGEOMETRY;
+ }
+
+ @Override
+ public boolean deepEqual(IAObject obj) {
+ if (!(obj instanceof AGeometry)) {
+ return false;
+ } else {
+ AGeometry p = (AGeometry) obj;
+ return p.geometry.equals(geometry);
+ }
+ }
+
+ @Override
+ public int hash() {
+ return geometry.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return geometry.toString();
+ }
+
+ @Override
+ public ObjectNode toJSON() {
+ ObjectMapper om = new ObjectMapper();
+ ObjectNode json = null;
+ try {
+ json = (ObjectNode) om.readTree(geometry.asGeoJson());
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return json;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.java
new file mode 100644
index 0000000..346d68a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableGeometry.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.om.base;
+
+import com.esri.core.geometry.OGCStructure;
+import com.esri.core.geometry.OperatorImportFromWkt;
+import com.esri.core.geometry.SpatialReference;
+import com.esri.core.geometry.WktImportFlags;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class AMutableGeometry extends AGeometry {
+
+ private OperatorImportFromWkt wktImporter;
+
+ public AMutableGeometry(OGCGeometry geom) {
+ super(geom);
+ wktImporter = OperatorImportFromWkt.local();
+ }
+
+ public void setValue(OGCGeometry geom) {
+ this.geometry = geom;
+ }
+
+ public void parseWKT(String wkt) {
+ OGCStructure structure;
+
+ structure = wktImporter.executeOGC(WktImportFlags.wktImportNonTrusted, wkt, null);
+ this.geometry = OGCGeometry.createFromOGCStructure(structure, SpatialReference.create(4326));
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
index 2fb69ab..9a097dc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/AMutableUUID.java
@@ -23,15 +23,16 @@
public class AMutableUUID extends AUUID {
- public void parseUUIDString(String tokenImage) throws HyracksDataException {
- if (tokenImage.length() != UUID_CHARS) {
- throw new HyracksDataException("This is not a correct UUID value: " + tokenImage);
+ private final byte[] hexBytesBuffer = new byte[UUID_CHARS];
+
+ public void parseUUIDString(char[] buffer, int begin, int len) throws HyracksDataException {
+ if (len != UUID_CHARS) {
+ throw new HyracksDataException("This is not a correct UUID value: " + new String(buffer, begin, len));
}
- byte[] hexBytes = new byte[UUID_CHARS];
- for (int i = 0; i < tokenImage.length(); i++) {
- hexBytes[i] = (byte) tokenImage.charAt(i);
+ for (int i = 0; i < len; i++) {
+ hexBytesBuffer[i] = (byte) buffer[begin + i];
}
- parseUUIDHexBytes(hexBytes, 0);
+ parseUUIDHexBytes(hexBytesBuffer, 0);
}
public void parseUUIDHexBytes(byte[] serString, int offset) throws HyracksDataException {
@@ -122,4 +123,4 @@
}
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
index 291d963..39f5b3a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
@@ -63,7 +63,6 @@
* @throws HyracksDataException
*/
public static int parseTimePart(String timeString, int start, int length) throws HyracksDataException {
-
int offset = 0;
int hour = 0, min = 0, sec = 0, millis = 0;
@@ -518,4 +517,4 @@
return timezone;
}
-}
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
index f837e36..98abb97 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/DateTimeFormatUtils.java
@@ -148,20 +148,20 @@
};
private static final byte[][] TIMEZONE_IDS;
- private static final int[] TIMEZONE_OFFSETS;
+ private static final TimeZone[] TIMEZONE_VALUES;
static {
String[] tzIds = TimeZone.getAvailableIDs();
int tzCount = tzIds.length;
TIMEZONE_IDS = new byte[tzCount][];
- TIMEZONE_OFFSETS = new int[tzCount];
+ TIMEZONE_VALUES = new TimeZone[tzCount];
for (int i = 0; i < tzCount; i++) {
TIMEZONE_IDS[i] = tzIds[i].getBytes(ENCODING);
}
Arrays.sort(TIMEZONE_IDS, byteArrayComparator);
for (int i = 0; i < tzCount; i++) {
- TIMEZONE_OFFSETS[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING)).getRawOffset();
+ TIMEZONE_VALUES[i] = TimeZone.getTimeZone(new String(TIMEZONE_IDS[i], ENCODING));
}
}
@@ -241,9 +241,10 @@
return -1;
}
- private int binaryTimezoneIDSearch(byte[] barray, int start, int length) {
- return Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
+ public static TimeZone findTimeZone(byte[] barray, int start, int length) {
+ int idx = Arrays.binarySearch(TIMEZONE_IDS, 0, TIMEZONE_IDS.length,
Arrays.copyOfRange(barray, start, start + length), byteArrayComparator);
+ return idx >= 0 ? TIMEZONE_VALUES[idx] : null;
}
private int indexOf(byte[] barray, int start, int length, char c) {
@@ -683,10 +684,10 @@
|| data[dataStart + timezoneEndField] == '_')) {
timezoneEndField++;
}
- int searchIdx = binaryTimezoneIDSearch(data, dataStart + dataStringPointer,
- timezoneEndField - dataStringPointer);
- if (searchIdx >= 0) {
- timezone = TIMEZONE_OFFSETS[searchIdx];
+ TimeZone tz =
+ findTimeZone(data, dataStart + dataStringPointer, timezoneEndField - dataStringPointer);
+ if (tz != null) {
+ timezone = tz.getRawOffset();
} else {
if (raiseParseDataError) {
throw new AsterixTemporalTypeParseException("Unexpected timezone string: " + new String(
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 8cd18fc..c40d550 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -36,6 +36,7 @@
import org.apache.asterix.om.typecomputer.impl.ADoubleTypeComputer;
import org.apache.asterix.om.typecomputer.impl.ADurationTypeComputer;
import org.apache.asterix.om.typecomputer.impl.AFloatTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.AGeometryTypeComputer;
import org.apache.asterix.om.typecomputer.impl.AInt16TypeComputer;
import org.apache.asterix.om.typecomputer.impl.AInt32TypeComputer;
import org.apache.asterix.om.typecomputer.impl.AInt64TypeComputer;
@@ -61,6 +62,7 @@
import org.apache.asterix.om.typecomputer.impl.CollectionMemberResultType;
import org.apache.asterix.om.typecomputer.impl.CollectionToSequenceTypeComputer;
import org.apache.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.DoubleIfTypeComputer;
import org.apache.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
import org.apache.asterix.om.typecomputer.impl.FieldAccessByNameResultType;
import org.apache.asterix.om.typecomputer.impl.FieldAccessNestedResultType;
@@ -73,11 +75,14 @@
import org.apache.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
import org.apache.asterix.om.typecomputer.impl.LocalAvgTypeComputer;
import org.apache.asterix.om.typecomputer.impl.MinMaxAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.MissingIfTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NonTaggedGetItemResultType;
import org.apache.asterix.om.typecomputer.impl.NotUnknownTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NullIfTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NullableDoubleTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericAddSubMulDivTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericAggTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.NumericDivideTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericDoubleOutputFunctionTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericInt8OutputFunctionTypeComputer;
import org.apache.asterix.om.typecomputer.impl.NumericRound2TypeComputer;
@@ -213,6 +218,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-names", 1);
public static final FunctionIdentifier RECORD_PAIRS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "object-pairs", FunctionIdentifier.VARARGS);
+ public static final FunctionIdentifier GEOMETRY_CONSTRUCTOR =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-geojson", FunctionIdentifier.VARARGS);
// numeric
public static final FunctionIdentifier NUMERIC_UNARY_MINUS =
@@ -225,9 +232,10 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-divide", 2);
public static final FunctionIdentifier NUMERIC_MOD =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-mod", 2);
- public static final FunctionIdentifier NUMERIC_IDIV =
- new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-idiv", 2);
- public static final FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
+ public static final FunctionIdentifier NUMERIC_DIV =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "numeric-div", 2);
+ public static final FunctionIdentifier NUMERIC_POWER =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "power", 2);
public static final FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "abs", 1);
public static final FunctionIdentifier NUMERIC_ACOS =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "acos", 1);
@@ -237,6 +245,10 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan", 1);
public static final FunctionIdentifier NUMERIC_ATAN2 =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "atan2", 2);
+ public static final FunctionIdentifier NUMERIC_DEGREES =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "degrees", 1);
+ public static final FunctionIdentifier NUMERIC_RADIANS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "radians", 1);
public static final FunctionIdentifier NUMERIC_COS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cos", 1);
public static final FunctionIdentifier NUMERIC_SIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sin", 1);
public static final FunctionIdentifier NUMERIC_TAN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tan", 1);
@@ -247,6 +259,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sqrt", 1);
public static final FunctionIdentifier NUMERIC_SIGN =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sign", 1);
+ public static final FunctionIdentifier NUMERIC_E = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "e", 0);
+ public static final FunctionIdentifier NUMERIC_PI = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "pi", 0);
public static final FunctionIdentifier NUMERIC_CEILING =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "ceiling", 1);
@@ -321,6 +335,8 @@
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 3);
public static final FunctionIdentifier STRING_REPLACE_WITH_LIMIT =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "replace", 4);
+ public static final FunctionIdentifier STRING_REVERSE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reverse", 1);
public static final FunctionIdentifier STRING_LENGTH =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "string-length", 1);
public static final FunctionIdentifier STRING_LIKE =
@@ -733,6 +749,129 @@
public static final FunctionIdentifier UUID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "uuid", 0);
public static final FunctionIdentifier CREATE_QUERY_UID =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-query-uid", 0);
+ public static final FunctionIdentifier RANDOM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 0);
+ public static final FunctionIdentifier RANDOM_WITH_SEED =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "random", 1);
+
+ //Geo
+ public static final FunctionIdentifier ST_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-area", 1);
+ public static final FunctionIdentifier ST_MAKE_POINT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 2);
+ public static final FunctionIdentifier ST_MAKE_POINT3D =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 3);
+ public static final FunctionIdentifier ST_MAKE_POINT3D_M =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-point", 4);
+ public static final FunctionIdentifier ST_INTERSECTS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersects", 2);
+ public static final FunctionIdentifier ST_UNION =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 2);
+ public static final FunctionIdentifier ST_IS_COLLECTION =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-collection", 1);
+ public static final FunctionIdentifier ST_CONTAINS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-contains", 2);
+ public static final FunctionIdentifier ST_CROSSES =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-crosses", 2);
+ public static final FunctionIdentifier ST_DISJOINT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-disjoint", 2);
+ public static final FunctionIdentifier ST_EQUALS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-equals", 2);
+ public static final FunctionIdentifier ST_OVERLAPS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-overlaps", 2);
+ public static final FunctionIdentifier ST_TOUCHES =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-touches", 2);
+ public static final FunctionIdentifier ST_WITHIN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-within", 2);
+ public static final FunctionIdentifier ST_IS_EMPTY =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-empty", 1);
+ public static final FunctionIdentifier ST_IS_SIMPLE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-simple", 1);
+ public static final FunctionIdentifier ST_COORD_DIM =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-coord-dim", 1);
+ public static final FunctionIdentifier ST_DIMENSION =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-dimension", 1);
+ public static final FunctionIdentifier GEOMETRY_TYPE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "geometry-type", 1);
+ public static final FunctionIdentifier ST_M = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-m", 1);
+ public static final FunctionIdentifier ST_N_RINGS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-rings", 1);
+ public static final FunctionIdentifier ST_N_POINTS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-n-points", 1);
+ public static final FunctionIdentifier ST_NUM_GEOMETRIIES =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-geometries", 1);
+ public static final FunctionIdentifier ST_NUM_INTERIOR_RINGS =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-num-interior-rings", 1);
+ public static final FunctionIdentifier ST_SRID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-srid", 1);
+ public static final FunctionIdentifier ST_X = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x", 1);
+ public static final FunctionIdentifier ST_Y = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y", 1);
+ public static final FunctionIdentifier ST_X_MAX =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-max", 1);
+ public static final FunctionIdentifier ST_X_MIN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-x-min", 1);
+ public static final FunctionIdentifier ST_Y_MAX =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-max", 1);
+ public static final FunctionIdentifier ST_Y_MIN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-y-min", 1);
+ public static final FunctionIdentifier ST_Z = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z", 1);
+ public static final FunctionIdentifier ST_Z_MIN =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-min", 1);
+ public static final FunctionIdentifier ST_Z_MAX =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-z-max", 1);
+ public static final FunctionIdentifier ST_AS_BINARY =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-binary", 1);
+ public static final FunctionIdentifier ST_AS_TEXT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-text", 1);
+ public static final FunctionIdentifier ST_AS_GEOJSON =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-as-geojson", 1);
+ public static final FunctionIdentifier ST_DISTANCE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-distance", 2);
+ public static final FunctionIdentifier ST_LENGTH =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-length", 1);
+ public static final FunctionIdentifier SCALAR_ST_UNION_AGG =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union", 1);
+ public static final FunctionIdentifier ST_UNION_AGG =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-union-agg", 1);
+ public static final FunctionIdentifier ST_GEOM_FROM_TEXT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 1);
+ public static final FunctionIdentifier ST_GEOM_FROM_TEXT_SRID =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-text", 2);
+ public static final FunctionIdentifier ST_GEOM_FROM_WKB =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 1);
+ public static final FunctionIdentifier ST_GEOM_FROM_WKB_SRID =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geom-from-wkb", 2);
+ public static final FunctionIdentifier ST_LINE_FROM_MULTIPOINT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-line-from-multipoint", 1);
+ public static final FunctionIdentifier ST_MAKE_ENVELOPE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-make-envelope", 5);
+ public static final FunctionIdentifier ST_IS_CLOSED =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-closed", 1);
+ public static final FunctionIdentifier ST_IS_RING =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-is-ring", 1);
+ public static final FunctionIdentifier ST_RELATE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-relate", 3);
+ public static final FunctionIdentifier ST_BOUNDARY =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-boundary", 1);
+ public static final FunctionIdentifier ST_END_POINT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-end-point", 1);
+ public static final FunctionIdentifier ST_ENVELOPE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-envelope", 1);
+ public static final FunctionIdentifier ST_EXTERIOR_RING =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-exterior-ring", 1);
+ public static final FunctionIdentifier ST_GEOMETRY_N =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-geometry-n", 2);
+ public static final FunctionIdentifier ST_INTERIOR_RING_N =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-interior-ring-n", 2);
+ public static final FunctionIdentifier ST_POINT_N =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-point-n", 2);
+ public static final FunctionIdentifier ST_START_POINT =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-start-point", 1);
+ public static final FunctionIdentifier ST_DIFFERENCE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-difference", 2);
+ public static final FunctionIdentifier ST_INTERSECTION =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-intersection", 2);
+ public static final FunctionIdentifier ST_SYM_DIFFERENCE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-sym-difference", 2);
+ public static final FunctionIdentifier ST_POLYGONIZE =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-polygonize", 1);
// Spatial and temporal type accessors
public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR =
@@ -876,6 +1015,15 @@
public static final FunctionIdentifier IF_NAN_OR_INF =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "if-nan-or-inf", FunctionIdentifier.VARARGS);
+ public static final FunctionIdentifier MISSING_IF =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "missing-if", 2);
+ public static final FunctionIdentifier NULL_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "null-if", 2);
+ public static final FunctionIdentifier NAN_IF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "nan-if", 2);
+ public static final FunctionIdentifier POSINF_IF =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "posinf-if", 2);
+ public static final FunctionIdentifier NEGINF_IF =
+ new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "neginf-if", 2);
+
public static final FunctionIdentifier TO_ATOMIC =
new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-atomic", 1);
public static final FunctionIdentifier TO_ARRAY =
@@ -944,7 +1092,6 @@
addPrivateFunction(CHECK_UNKNOWN, NotUnknownTypeComputer.INSTANCE, true);
addPrivateFunction(ANY_COLLECTION_MEMBER, CollectionMemberResultType.INSTANCE, true);
addFunction(BOOLEAN_CONSTRUCTOR, ABooleanTypeComputer.INSTANCE, true);
- addFunction(CARET, NumericAddSubMulDivTypeComputer.INSTANCE, true);
addFunction(CIRCLE_CONSTRUCTOR, ACircleTypeComputer.INSTANCE, true);
addPrivateFunction(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE, true);
@@ -960,6 +1107,8 @@
addFunction(UUID, AUUIDTypeComputer.INSTANCE, false);
addFunction(CREATE_QUERY_UID, ABinaryTypeComputer.INSTANCE, false);
addFunction(UUID_CONSTRUCTOR, AUUIDTypeComputer.INSTANCE, true);
+ addFunction(RANDOM, ADoubleTypeComputer.INSTANCE, false);
+ addFunction(RANDOM_WITH_SEED, ADoubleTypeComputer.INSTANCE, false);
addFunction(DATE_CONSTRUCTOR, ADateTypeComputer.INSTANCE, true);
addFunction(DATETIME_CONSTRUCTOR, ADateTimeTypeComputer.INSTANCE, true);
@@ -981,9 +1130,6 @@
addPrivateFunction(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE, true);
addPrivateFunction(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
addPrivateFunction(HASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE, true);
- addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
- addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
- addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
addPrivateFunction(INDEX_SEARCH, AnyTypeComputer.INSTANCE, true);
addFunction(INT8_CONSTRUCTOR, AInt8TypeComputer.INSTANCE, true);
addFunction(INT16_CONSTRUCTOR, AInt16TypeComputer.INSTANCE, true);
@@ -998,20 +1144,25 @@
addPrivateFunction(NUMERIC_UNARY_MINUS, UnaryMinusTypeComputer.INSTANCE, true);
addPrivateFunction(NUMERIC_SUBTRACT, NumericAddSubMulDivTypeComputer.INSTANCE, true);
addPrivateFunction(NUMERIC_MULTIPLY, NumericAddSubMulDivTypeComputer.INSTANCE, true);
- addPrivateFunction(NUMERIC_DIVIDE, NumericAddSubMulDivTypeComputer.INSTANCE, true);
+ addPrivateFunction(NUMERIC_DIVIDE, NumericDivideTypeComputer.INSTANCE, true);
addPrivateFunction(NUMERIC_MOD, NumericAddSubMulDivTypeComputer.INSTANCE, true);
- addPrivateFunction(NUMERIC_IDIV, AInt64TypeComputer.INSTANCE, true);
+ addPrivateFunction(NUMERIC_DIV, NumericAddSubMulDivTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ABS, NumericUnaryFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ACOS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ASIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ATAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ATAN2, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+ addFunction(NUMERIC_DEGREES, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+ addFunction(NUMERIC_RADIANS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_COS, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_SIN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_TAN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+ addFunction(NUMERIC_E, ADoubleTypeComputer.INSTANCE, true);
addFunction(NUMERIC_EXP, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_LN, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_LOG, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
+ addFunction(NUMERIC_PI, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(NUMERIC_POWER, NumericAddSubMulDivTypeComputer.INSTANCE, true);
addFunction(NUMERIC_SQRT, NumericDoubleOutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_SIGN, NumericInt8OutputFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_CEILING, NumericUnaryFunctionTypeComputer.INSTANCE, true);
@@ -1020,9 +1171,6 @@
addFunction(NUMERIC_ROUND_HALF_TO_EVEN, NumericUnaryFunctionTypeComputer.INSTANCE, true);
addFunction(NUMERIC_ROUND_HALF_TO_EVEN2, NumericRound2TypeComputer.INSTANCE, true);
addFunction(NUMERIC_TRUNC, NumericRound2TypeComputer.INSTANCE, true);
- addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
- addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
- addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
addFunction(BINARY_LENGTH, UnaryBinaryInt64TypeComputer.INSTANCE, true);
addFunction(PARSE_BINARY, ABinaryTypeComputer.INSTANCE, true);
@@ -1063,6 +1211,7 @@
addFunction(STRING_REGEXP_REPLACE_WITH_FLAG, StringStringTypeComputer.INSTANCE, true);
addFunction(STRING_REPLACE, StringStringTypeComputer.INSTANCE, true);
addFunction(STRING_REPLACE_WITH_LIMIT, StringIntToStringTypeComputer.INSTANCE_TRIPLE_STRING, true);
+ addFunction(STRING_REVERSE, StringStringTypeComputer.INSTANCE, true);
addFunction(SUBSTRING_BEFORE, StringStringTypeComputer.INSTANCE, true);
addFunction(SUBSTRING_AFTER, StringStringTypeComputer.INSTANCE, true);
addPrivateFunction(STRING_EQUAL, StringBooleanTypeComputer.INSTANCE, true);
@@ -1087,6 +1236,19 @@
addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
+ addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+ addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
+ addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
+ addFunction(IF_NULL, IfNullTypeComputer.INSTANCE, true);
+ addFunction(IF_NAN, IfNanOrInfTypeComputer.INSTANCE, true);
+ addFunction(IF_NAN_OR_INF, IfNanOrInfTypeComputer.INSTANCE, true);
+
+ addFunction(MISSING_IF, MissingIfTypeComputer.INSTANCE, true);
+ addFunction(NULL_IF, NullIfTypeComputer.INSTANCE, true);
+ addFunction(NAN_IF, DoubleIfTypeComputer.INSTANCE, true);
+ addFunction(POSINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+ addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
+
// Aggregate Functions
addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
addPrivateFunction(LOCAL_MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1198,6 +1360,71 @@
addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE, true);
addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE, true);
+ //geo functions
+ addFunction(ST_AREA, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_MAKE_POINT, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_MAKE_POINT3D, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_MAKE_POINT3D_M, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_INTERSECTS, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_UNION, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_CONTAINS, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_CROSSES, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_DISJOINT, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_EQUALS, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_OVERLAPS, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_TOUCHES, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_WITHIN, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_EMPTY, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_SIMPLE, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_COLLECTION, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_COORD_DIM, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_DIMENSION, AInt32TypeComputer.INSTANCE, true);
+ addFunction(GEOMETRY_TYPE, AStringTypeComputer.INSTANCE, true);
+ addFunction(ST_M, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_N_RINGS, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_N_POINTS, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_NUM_GEOMETRIIES, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_NUM_INTERIOR_RINGS, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_SRID, AInt32TypeComputer.INSTANCE, true);
+ addFunction(ST_X, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Y, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_X_MAX, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_X_MIN, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Y_MAX, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Y_MIN, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Z, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Z_MIN, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_Z_MAX, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_AS_BINARY, ABinaryTypeComputer.INSTANCE, true);
+ addFunction(ST_AS_TEXT, AStringTypeComputer.INSTANCE, true);
+ addFunction(ST_AS_GEOJSON, AStringTypeComputer.INSTANCE, true);
+ addFunction(ST_DISTANCE, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_LENGTH, ADoubleTypeComputer.INSTANCE, true);
+ addFunction(ST_GEOM_FROM_TEXT, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_GEOM_FROM_TEXT_SRID, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_GEOM_FROM_WKB, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_GEOM_FROM_WKB_SRID, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_LINE_FROM_MULTIPOINT, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_MAKE_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_CLOSED, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_IS_RING, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_RELATE, ABooleanTypeComputer.INSTANCE, true);
+ addFunction(ST_BOUNDARY, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_END_POINT, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_ENVELOPE, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_EXTERIOR_RING, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_GEOMETRY_N, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_INTERIOR_RING_N, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_POINT_N, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_START_POINT, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_INTERSECTION, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_SYM_DIFFERENCE, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(SCALAR_ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+ addPrivateFunction(ST_UNION_AGG, AGeometryTypeComputer.INSTANCE, true);
+ addFunction(ST_POLYGONIZE, AGeometryTypeComputer.INSTANCE, true);
+
// Binary functions
addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE, true);
@@ -1232,6 +1459,7 @@
addFunction(RECORD_LENGTH, AInt64TypeComputer.INSTANCE_NULLABLE, true);
addFunction(RECORD_NAMES, OrderedListOfAStringTypeComputer.INSTANCE_NULLABLE, true);
addFunction(RECORD_PAIRS, RecordPairsTypeComputer.INSTANCE, true);
+ addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
// temporal type accessors
addFunction(ACCESSOR_TEMPORAL_YEAR, AInt64TypeComputer.INSTANCE, true);
@@ -1563,6 +1791,13 @@
addDistinctAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM);
addScalarAgg(SQL_SUM_DISTINCT, SCALAR_SQL_SUM_DISTINCT);
+
+ // SPATIAL AGGREGATES
+
+ addAgg(ST_UNION_AGG);
+ addLocalAgg(ST_UNION_AGG, ST_UNION_AGG);
+ addIntermediateAgg(ST_UNION_AGG, ST_UNION_AGG);
+ addGlobalAgg(ST_UNION_AGG, ST_UNION_AGG);
}
static {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.java
new file mode 100644
index 0000000..e85410ab5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AGeometryTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class AGeometryTypeComputer extends AbstractResultTypeComputer {
+
+ public static final AGeometryTypeComputer INSTANCE = new AGeometryTypeComputer();
+
+ private AGeometryTypeComputer() {
+ }
+
+ @Override
+ protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ return BuiltinType.AGEOMETRY;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.java
new file mode 100644
index 0000000..105ea52
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/DoubleIfTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class DoubleIfTypeComputer extends AbstractResultTypeComputer {
+
+ public static final DoubleIfTypeComputer INSTANCE = new DoubleIfTypeComputer();
+
+ @Override
+ public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ IAType inputType = strippedInputTypes[0];
+ return inputType.getTypeTag() == ATypeTag.DOUBLE ? inputType : BuiltinType.ANY;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.java
new file mode 100644
index 0000000..bb5203e
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MissingIfTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class MissingIfTypeComputer extends AbstractResultTypeComputer {
+
+ public static final MissingIfTypeComputer INSTANCE = new MissingIfTypeComputer();
+
+ @Override
+ public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ IAType inputType = strippedInputTypes[0];
+ return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createMissableType(inputType);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.java
new file mode 100644
index 0000000..590d8f6
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NullIfTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class NullIfTypeComputer extends AbstractResultTypeComputer {
+
+ public static final NullIfTypeComputer INSTANCE = new NullIfTypeComputer();
+
+ @Override
+ public IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ IAType inputType = strippedInputTypes[0];
+ return inputType.getTypeTag() == ATypeTag.ANY ? inputType : AUnionType.createNullableType(inputType);
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
new file mode 100644
index 0000000..92d3bd5
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericDivideTypeComputer.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.IncompatibleTypeException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+
+/**
+ * Returns double if both operands are integers
+ */
+public class NumericDivideTypeComputer extends AbstractResultTypeComputer {
+ public static final NumericDivideTypeComputer INSTANCE = new NumericDivideTypeComputer();
+
+ private NumericDivideTypeComputer() {
+ }
+
+ @Override
+ protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+ AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expr;
+ String funcName = functionCallExpression.getFunctionIdentifier().getName();
+ IAType t1 = strippedInputTypes[0];
+ IAType t2 = strippedInputTypes[1];
+ ATypeTag tag1 = t1.getTypeTag();
+ ATypeTag tag2 = t2.getTypeTag();
+
+ IAType type;
+ switch (tag1) {
+ case DOUBLE:
+ switch (tag2) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ type = BuiltinType.ADOUBLE;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case FLOAT:
+ switch (tag2) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ type = BuiltinType.AFLOAT;
+ break;
+ case DOUBLE:
+ type = BuiltinType.ADOUBLE;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case BIGINT:
+ case INTEGER:
+ case SMALLINT:
+ case TINYINT:
+ switch (tag2) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case DOUBLE:
+ type = BuiltinType.ADOUBLE;
+ break;
+ case FLOAT:
+ type = BuiltinType.AFLOAT;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case ANY:
+ switch (tag2) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case ANY:
+ case DOUBLE:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case DATE:
+ switch (tag2) {
+ case DATE:
+ type = BuiltinType.ADURATION;
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ type = BuiltinType.ADATE;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case TIME:
+ switch (tag2) {
+ case TIME:
+ type = BuiltinType.ADURATION;
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ type = BuiltinType.ATIME;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case DATETIME:
+ switch (tag2) {
+ case DATETIME:
+ type = BuiltinType.ADURATION;
+ break;
+ case YEARMONTHDURATION:
+ case DAYTIMEDURATION:
+ case DURATION:
+ type = BuiltinType.ADATETIME;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case DURATION:
+ switch (tag2) {
+ case DATE:
+ type = BuiltinType.ADATE;
+ break;
+ case TIME:
+ type = BuiltinType.ATIME;
+ break;
+ case DATETIME:
+ type = BuiltinType.ADATETIME;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case YEARMONTHDURATION:
+ switch (tag2) {
+ case DATE:
+ type = BuiltinType.ADATE;
+ break;
+ case TIME:
+ type = BuiltinType.ATIME;
+ break;
+ case DATETIME:
+ type = BuiltinType.ADATETIME;
+ break;
+ case YEARMONTHDURATION:
+ type = BuiltinType.AYEARMONTHDURATION;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ case DAYTIMEDURATION:
+ switch (tag2) {
+ case DATE:
+ type = BuiltinType.ADATE;
+ break;
+ case TIME:
+ type = BuiltinType.ATIME;
+ break;
+ case DATETIME:
+ type = BuiltinType.ADATETIME;
+ break;
+ case DAYTIMEDURATION:
+ type = BuiltinType.ADAYTIMEDURATION;
+ break;
+ case ANY:
+ type = BuiltinType.ANY;
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ break;
+ default:
+ throw new IncompatibleTypeException(funcName, tag1, tag2);
+ }
+ return type;
+ }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
index baaed59..26cbf1f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ARecordType.java
@@ -28,7 +28,10 @@
import org.apache.asterix.common.annotations.IRecordTypeAnnotation;
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.om.base.IAObject;
+import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -226,6 +229,39 @@
}
/**
+ *
+ * @param subFieldName
+ * The full pathname of the field
+ * @return The nullability of the field
+ * @throws AlgebricksException
+ */
+ public boolean isSubFieldNullable(List<String> subFieldName) throws AlgebricksException {
+ IAType subRecordType = getFieldType(subFieldName.get(0));
+ for (int i = 1; i < subFieldName.size(); i++) {
+ if (subRecordType == null) {
+ // open field is nullable
+ return true;
+ }
+ if (subRecordType.getTypeTag().equals(ATypeTag.UNION)) {
+ if (NonTaggedFormatUtil.isOptional(subRecordType)) {
+ return true;
+ }
+ subRecordType = ((AUnionType) subRecordType).getActualType();
+ if (subRecordType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new AsterixException(
+ "Field accessor is not defined for values of type " + subRecordType.getTypeTag());
+ }
+ }
+ if (!(subRecordType instanceof ARecordType)) {
+ throw CompilationException.create(ErrorCode.COMPILATION_ILLEGAL_STATE,
+ "Illegal field type " + subRecordType.getTypeTag() + " when checking field nullability");
+ }
+ subRecordType = ((ARecordType) subRecordType).getFieldType(subFieldName.get(i));
+ }
+ return subRecordType == null || NonTaggedFormatUtil.isOptional(subRecordType);
+ }
+
+ /**
* Returns the field type of the field name if it exists, otherwise null.
*
* @param fieldName
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
index 825ed70..a79588d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/ATypeTag.java
@@ -66,7 +66,8 @@
DAYTIMEDURATION(37),
UUID(38),
SHORTWITHOUTTYPEINFO(40),
- NULL(41);
+ NULL(41),
+ GEOMETRY(42);
/*
* Serialized Tags begin
@@ -100,6 +101,7 @@
public static final byte SERIALIZED_FLOAT_TYPE_TAG = FLOAT.serialize();
public static final byte SERIALIZED_BINARY_TYPE_TAG = BINARY.serialize();
public static final byte SERIALIZED_UUID_TYPE_TAG = UUID.serialize();
+ public static final byte SERIALIZED_GEOMETRY_TYPE_TAG = GEOMETRY.serialize();
/*
* Serialized Tags end
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
index 3037e7c..a36e0e4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/BuiltinType.java
@@ -682,6 +682,33 @@
}
};
+ public final static BuiltinType AGEOMETRY = new LowerCaseConstructorType() {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public String getDisplayName() {
+ return "AGEOMETRY";
+ }
+
+ @Override
+ public ATypeTag getTypeTag() {
+ return ATypeTag.GEOMETRY;
+ }
+
+ @Override
+ public String getTypeName() {
+ return "geometry";
+ }
+
+ @Override
+ public ObjectNode toJSON() {
+ ObjectNode type = new ObjectMapper().createObjectNode();
+ type.put("type", "AGEOMETRY");
+ return type;
+ }
+ };
+
public final static BuiltinType ACIRCLE = new LowerCaseConstructorType() {
private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
index cfc1b55..254dbee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/types/TypeTagUtil.java
@@ -88,6 +88,8 @@
return AUnorderedListType.FULLY_OPEN_UNORDEREDLIST_TYPE;
case ARRAY:
return AOrderedListType.FULL_OPEN_ORDEREDLIST_TYPE;
+ case GEOMETRY:
+ return BuiltinType.AGEOMETRY;
default:
// TODO(tillw) should be an internal error
throw new HyracksDataException("Typetag " + typeTag + " is not a built-in type");
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
index 8558538..7da0263 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
@@ -21,6 +21,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
@@ -68,6 +69,7 @@
case ARRAY:
case MULTISET:
case POLYGON:
+ case GEOMETRY:
case ANY:
return false;
default:
@@ -193,6 +195,14 @@
} else {
return AUnorderedListSerializerDeserializer.getUnorderedListLength(serNonTaggedAObject, offset) - 1;
}
+ case GEOMETRY:
+ // Since Geometry is variable size, we store its size at the first 32 bits for efficiency
+ // @see: STGeomFromTextDescriptor#createEvaluatorFactory, AGeometrySerializerDeserializer#serialize
+ if (tagged) {
+ return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset + 1) + 4;
+ } else {
+ return AInt32SerializerDeserializer.getInt(serNonTaggedAObject, offset) + 4;
+ }
default:
throw new NotImplementedException(
"No getLength implemented for a value of this type " + typeTag + " .");
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
index a94f073..2ed2ac9 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/logging/ReplicationLogBuffer.java
@@ -57,7 +57,7 @@
}
}
- public synchronized void isFull(boolean full) {
+ public synchronized void setFull(boolean full) {
this.full.set(full);
this.notify();
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
index 7a9d75f..6c8e372 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/LogReplicationManager.java
@@ -167,7 +167,7 @@
private synchronized void appendToLogBuffer(ILogRecord logRecord) throws InterruptedException {
if (!currentTxnLogBuffer.hasSpace(logRecord)) {
- currentTxnLogBuffer.isFull(true);
+ currentTxnLogBuffer.setFull(true);
if (logRecord.getLogSize() > logPageSize) {
getAndInitNewLargePage(logRecord.getLogSize());
} else {
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
index 84922cd..d4d601c 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/CheckpointPartitionIndexesTask.java
@@ -22,16 +22,20 @@
import java.io.DataOutputStream;
import java.io.IOException;
import java.io.OutputStream;
+import java.nio.file.Path;
import java.util.Collection;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.common.storage.DatasetResourceReference;
import org.apache.asterix.common.storage.IIndexCheckpointManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
+import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.IIOManager;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
import org.apache.hyracks.storage.common.LocalResource;
/**
@@ -51,13 +55,27 @@
appCtx.getIndexCheckpointManagerProvider();
PersistentLocalResourceRepository resRepo =
(PersistentLocalResourceRepository) appCtx.getLocalResourceRepository();
+ final IIOManager ioManager = appCtx.getIoManager();
final Collection<LocalResource> partitionResources = resRepo.getPartitionResources(partition).values();
final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
for (LocalResource ls : partitionResources) {
- final IIndexCheckpointManager indexCheckpointManager =
- indexCheckpointManagerProvider.get(DatasetResourceReference.of(ls));
+ DatasetResourceReference ref = DatasetResourceReference.of(ls);
+ final IIndexCheckpointManager indexCheckpointManager = indexCheckpointManagerProvider.get(ref);
indexCheckpointManager.delete();
- indexCheckpointManager.init(currentLSN);
+ // Get most recent timestamp of existing files to avoid deletion
+ Path indexPath = StoragePathUtil.getIndexPath(ioManager, ref);
+ String[] files = indexPath.toFile().list(AbstractLSMIndexFileManager.COMPONENT_FILES_FILTER);
+ if (files == null) {
+ throw HyracksDataException
+ .create(new IOException(indexPath + " is not a directory or an IO Error occurred"));
+ }
+ String mostRecentTimestamp = null;
+ for (String file : files) {
+ String nextTimeStamp = AbstractLSMIndexFileManager.getComponentEndTime(file);
+ mostRecentTimestamp = mostRecentTimestamp == null || nextTimeStamp.compareTo(mostRecentTimestamp) > 0
+ ? nextTimeStamp : mostRecentTimestamp;
+ }
+ indexCheckpointManager.init(mostRecentTimestamp, currentLSN);
}
ReplicationProtocol.sendAck(worker.getChannel(), worker.getReusableBuffer());
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
index 57474ef..a4f9b43 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/MarkComponentValidTask.java
@@ -34,6 +34,7 @@
import org.apache.asterix.common.storage.ResourceReference;
import org.apache.asterix.replication.api.IReplicaTask;
import org.apache.asterix.replication.api.IReplicationWorker;
+import org.apache.asterix.replication.sync.IndexSynchronizer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexFileManager;
@@ -43,17 +44,21 @@
public class MarkComponentValidTask implements IReplicaTask {
private final long masterLsn;
+ private final long lastComponentId;
private final String file;
- public MarkComponentValidTask(String file, long masterLsn) {
+ public MarkComponentValidTask(String file, long masterLsn, long lastComponentId) {
this.file = file;
+ this.lastComponentId = lastComponentId;
this.masterLsn = masterLsn;
}
@Override
public void perform(INcApplicationContext appCtx, IReplicationWorker worker) {
try {
- if (masterLsn > 0) {
+ if (masterLsn == IndexSynchronizer.BULKLOAD_LSN) {
+ updateBulkLoadedLastComponentTimestamp(appCtx);
+ } else if (masterLsn != IndexSynchronizer.MERGE_LSN) {
ensureComponentLsnFlushed(appCtx);
}
// delete mask
@@ -65,6 +70,15 @@
}
}
+ private void updateBulkLoadedLastComponentTimestamp(INcApplicationContext appCtx) throws HyracksDataException {
+ final ResourceReference indexRef = ResourceReference.of(file);
+ final IIndexCheckpointManagerProvider checkpointManagerProvider = appCtx.getIndexCheckpointManagerProvider();
+ final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
+ final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
+ indexCheckpointManager.advanceValidComponentTimestamp(componentEndTime);
+
+ }
+
private void ensureComponentLsnFlushed(INcApplicationContext appCtx)
throws HyracksDataException, InterruptedException {
final ResourceReference indexRef = ResourceReference.of(file);
@@ -82,7 +96,7 @@
replicationTimeOut -= TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime);
}
final String componentEndTime = AbstractLSMIndexFileManager.getComponentEndTime(indexRef.getName());
- indexCheckpointManager.replicated(componentEndTime, masterLsn);
+ indexCheckpointManager.replicated(componentEndTime, masterLsn, lastComponentId);
}
}
@@ -97,6 +111,7 @@
final DataOutputStream dos = new DataOutputStream(out);
dos.writeUTF(file);
dos.writeLong(masterLsn);
+ dos.writeLong(lastComponentId);
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -105,6 +120,7 @@
public static MarkComponentValidTask create(DataInput input) throws IOException {
final String indexFile = input.readUTF();
final long lsn = input.readLong();
- return new MarkComponentValidTask(indexFile, lsn);
+ final long lastComponentId = input.readLong();
+ return new MarkComponentValidTask(indexFile, lsn, lastComponentId);
}
}
\ No newline at end of file
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
index ca0fcca..20663d1 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/messaging/ReplicateFileTask.java
@@ -30,12 +30,12 @@
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.exceptions.ReplicationException;
-import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.common.storage.IIndexCheckpointManager;
import org.apache.asterix.common.storage.IIndexCheckpointManagerProvider;
import org.apache.asterix.common.storage.ResourceReference;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.replication.api.IReplicaTask;
+import org.apache.asterix.replication.api.IReplicationWorker;
import org.apache.asterix.replication.management.NetworkingUtil;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
@@ -98,7 +98,7 @@
final IIndexCheckpointManager indexCheckpointManager = checkpointManagerProvider.get(indexRef);
final long currentLSN = appCtx.getTransactionSubsystem().getLogManager().getAppendLSN();
indexCheckpointManager.delete();
- indexCheckpointManager.init(currentLSN);
+ indexCheckpointManager.init(null, currentLSN);
LOGGER.info(() -> "Checkpoint index: " + indexRef);
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
index 95ae690..30a5595 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/sync/IndexSynchronizer.java
@@ -24,7 +24,7 @@
import java.io.IOException;
import org.apache.asterix.common.api.INcApplicationContext;
-import org.apache.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.replication.api.PartitionReplica;
import org.apache.asterix.replication.messaging.ComponentMaskTask;
@@ -37,12 +37,15 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexReplicationJob;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
public class IndexSynchronizer {
private static final Logger LOGGER = LogManager.getLogger();
+ public static final long MERGE_LSN = -1;
+ public static final long BULKLOAD_LSN = -2;
private final IReplicationJob job;
private final INcApplicationContext appCtx;
@@ -91,7 +94,8 @@
final FileSynchronizer fileSynchronizer = new FileSynchronizer(appCtx, replica);
job.getJobFiles().stream().map(StoragePathUtil::getFileRelativePath).forEach(fileSynchronizer::replicate);
// send mark component valid
- MarkComponentValidTask markValidTask = new MarkComponentValidTask(indexFile, getReplicatedComponentLsn());
+ MarkComponentValidTask markValidTask =
+ new MarkComponentValidTask(indexFile, getReplicatedComponentLsn(), getReplicatedComponentId());
ReplicationProtocol.sendTo(replica, markValidTask);
ReplicationProtocol.waitForAck(replica);
LOGGER.debug("Replicated component ({}) to replica {}", indexFile, replica);
@@ -118,12 +122,28 @@
private long getReplicatedComponentLsn() throws HyracksDataException {
final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+ if (indexReplJob.getLSMOpType() == LSMOperationType.MERGE) {
+ return MERGE_LSN;
+ } else if (indexReplJob.getLSMOpType() == LSMOperationType.LOAD) {
+ return BULKLOAD_LSN;
+ }
+
if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
- return AbstractLSMIOOperationCallback.INVALID;
+ return LSMIOOperationCallback.INVALID_LSN;
}
final ILSMIndex lsmIndex = indexReplJob.getLSMIndex();
final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
- return ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+ return ((LSMIOOperationCallback) lsmIndex.getIOOperationCallback())
.getComponentLSN(ctx.getComponentsToBeReplicated());
}
+
+ private long getReplicatedComponentId() throws HyracksDataException {
+ final ILSMIndexReplicationJob indexReplJob = (ILSMIndexReplicationJob) job;
+ if (indexReplJob.getLSMOpType() != LSMOperationType.FLUSH) {
+ return -1L;
+ }
+ final ILSMIndexOperationContext ctx = indexReplJob.getLSMIndexOperationContext();
+ LSMComponentId id = (LSMComponentId) ctx.getComponentsToBeReplicated().get(0).getId();
+ return id.getMinId();
+ }
}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
index 2712a47..96ce843 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/base/AsterixTupleFilter.java
@@ -41,7 +41,7 @@
}
@Override
- public boolean accept(IFrameTupleReference tuple) throws Exception {
+ public boolean accept(IFrameTupleReference tuple) throws HyracksDataException {
eval.evaluate(tuple, p);
return boolInspector.getBooleanValue(p.getByteArray(), p.getStartOffset(), p.getLength());
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index 620c543..b8c5f19 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -20,18 +20,10 @@
import java.io.DataOutput;
-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.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
@@ -41,85 +33,38 @@
public abstract class AbstractComparisonEvaluator implements IScalarEvaluator {
- protected ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- protected DataOutput out = resultStorage.getDataOutput();
- protected TaggedValuePointable argLeft = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
- protected TaggedValuePointable argRight = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
- protected IPointable outLeft = VoidPointable.FACTORY.createPointable();
- protected IPointable outRight = VoidPointable.FACTORY.createPointable();
- protected IScalarEvaluator evalLeft;
- protected IScalarEvaluator evalRight;
- private ComparisonHelper ch = new ComparisonHelper();
+ protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ protected final DataOutput out = resultStorage.getDataOutput();
+ protected final TaggedValuePointable argLeft =
+ (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+ protected final TaggedValuePointable argRight =
+ (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+ protected final IPointable outLeft = VoidPointable.FACTORY.createPointable();
+ protected final IPointable outRight = VoidPointable.FACTORY.createPointable();
+ protected final IScalarEvaluator evalLeft;
+ protected final IScalarEvaluator evalRight;
+ private final ComparisonHelper ch = new ComparisonHelper();
- @SuppressWarnings("unchecked")
- protected ISerializerDeserializer<ABoolean> serde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
- @SuppressWarnings("unchecked")
- protected ISerializerDeserializer<ANull> nullSerde =
- SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
-
- public AbstractComparisonEvaluator(IScalarEvaluatorFactory evalLeftFactory,
- IScalarEvaluatorFactory evalRightFactory, IHyracksTaskContext context) throws HyracksDataException {
- this.evalLeft = evalLeftFactory.createScalarEvaluator(context);
- this.evalRight = evalRightFactory.createScalarEvaluator(context);
+ public AbstractComparisonEvaluator(IScalarEvaluator evalLeft, IScalarEvaluator evalRight) {
+ this.evalLeft = evalLeft;
+ this.evalRight = evalRight;
}
@Override
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
-
// Evaluates input args.
evalLeft.evaluate(tuple, argLeft);
evalRight.evaluate(tuple, argRight);
argLeft.getValue(outLeft);
argRight.getValue(outRight);
- // checks whether we can apply >, >=, <, and <= to the given type since
- // these operations cannot be defined for certain types.
- if (isTotallyOrderable()) {
- checkTotallyOrderable();
- }
-
- // Checks whether two types are comparable
- if (comparabilityCheck()) {
- // Two types can be compared
- int r = compareResults();
- ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
- serde.serialize(b, out);
- } else {
- // result:NULL - two types cannot be compared.
- nullSerde.serialize(ANull.NULL, out);
- }
- result.set(resultStorage);
+ evaluateImpl(result);
}
- protected abstract boolean isTotallyOrderable();
-
- protected abstract boolean getComparisonResult(int r);
-
- // checks whether we can apply >, >=, <, and <= operations to the given type since
- // these operations can not be defined for certain types.
- protected void checkTotallyOrderable() throws HyracksDataException {
- if (argLeft.getLength() != 0) {
- ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
- switch (typeTag) {
- case DURATION:
- case INTERVAL:
- case LINE:
- case POINT:
- case POINT3D:
- case POLYGON:
- case CIRCLE:
- case RECTANGLE:
- throw new UnsupportedTypeException(ComparisonHelper.COMPARISON, argLeft.getTag());
- default:
- return;
- }
- }
- }
+ protected abstract void evaluateImpl(IPointable result) throws HyracksDataException;
// checks whether two types are comparable
- protected boolean comparabilityCheck() {
+ boolean comparabilityCheck() {
// Checks whether two types are comparable or not
ATypeTag typeTag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
ATypeTag typeTag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag());
@@ -128,10 +73,8 @@
return ATypeHierarchy.isCompatible(typeTag1, typeTag2);
}
- protected int compareResults() throws HyracksDataException {
- int result = ch.compare(EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag()),
+ int compare() throws HyracksDataException {
+ return ch.compare(EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag()),
EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argRight.getTag()), outLeft, outRight);
- return result;
}
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
new file mode 100644
index 0000000..c267bab
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractIfEqualsEvaluator.java
@@ -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.
+ */
+
+package org.apache.asterix.runtime.evaluators.comparisons;
+
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractIfEqualsEvaluator extends AbstractComparisonEvaluator {
+
+ AbstractIfEqualsEvaluator(IScalarEvaluator evalLeft, IScalarEvaluator evalRight) {
+ super(evalLeft, evalRight);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ if (comparabilityCheck() && compare() == 0) {
+ resultStorage.reset();
+ writeEqualsResult();
+ result.set(resultStorage);
+ } else {
+ result.set(argLeft);
+ }
+ }
+
+ protected abstract void writeEqualsResult() throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.java
new file mode 100644
index 0000000..446fed7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/AbstractValueComparisonEvaluator.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.evaluators.comparisons;
+
+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.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public abstract class AbstractValueComparisonEvaluator extends AbstractComparisonEvaluator {
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ABoolean> serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ABOOLEAN);
+ @SuppressWarnings("unchecked")
+ protected ISerializerDeserializer<ANull> nullSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+ public AbstractValueComparisonEvaluator(IScalarEvaluator evalLeft, IScalarEvaluator evalRight) {
+ super(evalLeft, evalRight);
+ }
+
+ @Override
+ protected void evaluateImpl(IPointable result) throws HyracksDataException {
+ resultStorage.reset();
+
+ // checks whether we can apply >, >=, <, and <= to the given type since
+ // these operations cannot be defined for certain types.
+ if (isTotallyOrderable()) {
+ checkTotallyOrderable();
+ }
+
+ // Checks whether two types are comparable
+ if (comparabilityCheck()) {
+ // Two types can be compared
+ int r = compare();
+ ABoolean b = getComparisonResult(r) ? ABoolean.TRUE : ABoolean.FALSE;
+ serde.serialize(b, out);
+ } else {
+ // result:NULL - two types cannot be compared.
+ nullSerde.serialize(ANull.NULL, out);
+ }
+ result.set(resultStorage);
+ }
+
+ protected abstract boolean isTotallyOrderable();
+
+ protected abstract boolean getComparisonResult(int r);
+
+ // checks whether we can apply >, >=, <, and <= operations to the given type since
+ // these operations can not be defined for certain types.
+ protected void checkTotallyOrderable() throws HyracksDataException {
+ if (argLeft.getLength() != 0) {
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argLeft.getTag());
+ switch (typeTag) {
+ case DURATION:
+ case INTERVAL:
+ case LINE:
+ case POINT:
+ case POINT3D:
+ case POLYGON:
+ case CIRCLE:
+ case RECTANGLE:
+ throw new UnsupportedTypeException(ComparisonHelper.COMPARISON, argLeft.getTag());
+ default:
+ return;
+ }
+ }
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
index b277885..c4fafa1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/EqualsDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
index 2826279..6a41104 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
index e12dd71..c468aac 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/GreaterThanOrEqualsDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
index 7e97dfa..4eac444 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
index 947dea1..ad365d6 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/LessThanOrEqualsDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
new file mode 100644
index 0000000..5b4f5e6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/MissingIfEqualsDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMissing;
+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.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MissingIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new MissingIfEqualsDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.MISSING_IF;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractIfEqualsEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
+
+ @SuppressWarnings("unchecked")
+ final ISerializerDeserializer<AMissing> missingSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AMISSING);
+
+ @Override
+ protected void writeEqualsResult() throws HyracksDataException {
+ missingSerde.serialize(AMissing.MISSING, out);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
new file mode 100644
index 0000000..7c6b5b2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NanIfEqualsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+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.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NanIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NanIfEqualsDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NAN_IF;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractIfEqualsEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
+
+ final AMutableDouble equalsResult = new AMutableDouble(Double.NaN);
+
+ @SuppressWarnings("unchecked")
+ final ISerializerDeserializer<ADouble> doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+ @Override
+ protected void writeEqualsResult() throws HyracksDataException {
+ doubleSerde.serialize(equalsResult, out);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..1c40357
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NegInfIfEqualsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+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.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NegInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NegInfIfEqualsDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NEGINF_IF;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractIfEqualsEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
+
+ final AMutableDouble equalsResult = new AMutableDouble(Double.NEGATIVE_INFINITY);
+
+ @SuppressWarnings("unchecked")
+ final ISerializerDeserializer<ADouble> doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+ @Override
+ protected void writeEqualsResult() throws HyracksDataException {
+ doubleSerde.serialize(equalsResult, out);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
index bfd5c98..17515af 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NotEqualsDescriptor.java
@@ -50,7 +50,8 @@
@Override
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
- return new AbstractComparisonEvaluator(args[0], args[1], ctx) {
+ return new AbstractValueComparisonEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
@Override
protected boolean getComparisonResult(int r) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
new file mode 100644
index 0000000..c09a75c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/NullIfEqualsDescriptor.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ANull;
+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.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NullIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NullIfEqualsDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NULL_IF;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractIfEqualsEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
+
+ @SuppressWarnings("unchecked")
+ final ISerializerDeserializer<ANull> nullSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ANULL);
+
+ @Override
+ protected void writeEqualsResult() throws HyracksDataException {
+ nullSerde.serialize(ANull.NULL, out);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
new file mode 100644
index 0000000..6a969b2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/comparisons/PosInfIfEqualsDescriptor.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.comparisons;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.ADouble;
+import org.apache.asterix.om.base.AMutableDouble;
+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.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class PosInfIfEqualsDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new PosInfIfEqualsDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.POSINF_IF;
+ }
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractIfEqualsEvaluator(args[0].createScalarEvaluator(ctx),
+ args[1].createScalarEvaluator(ctx)) {
+
+ final AMutableDouble equalsResult = new AMutableDouble(Double.POSITIVE_INFINITY);
+
+ @SuppressWarnings("unchecked")
+ final ISerializerDeserializer<ADouble> doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+ @Override
+ protected void writeEqualsResult() throws HyracksDataException {
+ doubleSerde.serialize(equalsResult, out);
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
index 936415d..b06c13a 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/AbstractNumericArithmeticEval.java
@@ -53,7 +53,6 @@
import org.apache.asterix.runtime.exceptions.TypeMismatchException;
import org.apache.asterix.runtime.exceptions.UnderflowException;
import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
@@ -95,8 +94,7 @@
abstract protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException;
@Override
- public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
- throws AlgebricksException {
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
return new IScalarEvaluatorFactory() {
private static final long serialVersionUID = 1L;
@@ -104,80 +102,102 @@
public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
return new IScalarEvaluator() {
- private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
- private DataOutput out = resultStorage.getDataOutput();
- private IPointable argPtr0 = new VoidPointable();
- private IPointable argPtr1 = new VoidPointable();
- private IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
- private IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
- private double[] operandsFloating = new double[args.length];
- private long[] operandsInteger = new long[args.length];
- private int resultType;
- static protected final int typeInt8 = 1;
- static protected final int typeInt16 = 2;
- static protected final int typeInt32 = 3;
- static protected final int typeInt64 = 4;
- static protected final int typeFloat = 5;
- static protected final int typeDouble = 6;
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput out = resultStorage.getDataOutput();
+ private final IPointable argPtr0 = new VoidPointable();
+ private final IPointable argPtr1 = new VoidPointable();
+ private final IScalarEvaluator evalLeft = args[0].createScalarEvaluator(ctx);
+ private final IScalarEvaluator evalRight = args[1].createScalarEvaluator(ctx);
+ private final double[] operandsFloating = new double[args.length];
+ private final long[] operandsInteger = new long[args.length];
- protected AMutableFloat aFloat = new AMutableFloat(0);
- protected AMutableDouble aDouble = new AMutableDouble(0);
- protected AMutableInt64 aInt64 = new AMutableInt64(0);
- protected AMutableInt32 aInt32 = new AMutableInt32(0);
- protected AMutableInt16 aInt16 = new AMutableInt16((short) 0);
- protected AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
+ private final AMutableFloat aFloat = new AMutableFloat(0);
+ private final AMutableDouble aDouble = new AMutableDouble(0);
+ private final AMutableInt64 aInt64 = new AMutableInt64(0);
+ private final AMutableInt32 aInt32 = new AMutableInt32(0);
+ private final AMutableInt16 aInt16 = new AMutableInt16((short) 0);
+ private final AMutableInt8 aInt8 = new AMutableInt8((byte) 0);
- protected AMutableDuration aDuration = new AMutableDuration(0, 0);
- protected AMutableDate aDate = new AMutableDate(0);
- protected AMutableTime aTime = new AMutableTime(0);
- protected AMutableDateTime aDatetime = new AMutableDateTime(0);
+ private final AMutableDuration aDuration = new AMutableDuration(0, 0);
+ private final AMutableDate aDate = new AMutableDate(0);
+ private final AMutableTime aTime = new AMutableTime(0);
+ private final AMutableDateTime aDatetime = new AMutableDateTime(0);
- private ATypeTag typeTag;
@SuppressWarnings("rawtypes")
- private ISerializerDeserializer serde;
+ private final ISerializerDeserializer int8Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT8);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer int16Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT16);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer int32Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer int64Serde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer floatSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer dateSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATE);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer timeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ATIME);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer dateTimeSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADATETIME);
+ @SuppressWarnings("rawtypes")
+ private final ISerializerDeserializer durationSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADURATION);
- @SuppressWarnings("unchecked")
@Override
+ @SuppressWarnings("unchecked")
public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
- resultStorage.reset();
- resultType = 0;
- int currentType;
evalLeft.evaluate(tuple, argPtr0);
evalRight.evaluate(tuple, argPtr1);
- for (int i = 0; i < args.length; i++) {
+ resultStorage.reset();
+
+ ATypeTag argTypeMax = null;
+
+ for (int i = 0; i < 2; i++) {
IPointable argPtr = i == 0 ? argPtr0 : argPtr1;
byte[] bytes = argPtr.getByteArray();
int offset = argPtr.getStartOffset();
- typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+ ATypeTag currentType;
+ ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
switch (typeTag) {
case TINYINT:
- currentType = typeInt8;
+ currentType = ATypeTag.TINYINT;
operandsInteger[i] = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
operandsFloating[i] = operandsInteger[i];
break;
case SMALLINT:
- currentType = typeInt16;
+ currentType = ATypeTag.SMALLINT;
operandsInteger[i] = AInt16SerializerDeserializer.getShort(bytes, offset + 1);
operandsFloating[i] = operandsInteger[i];
break;
case INTEGER:
- currentType = typeInt32;
+ currentType = ATypeTag.INTEGER;
operandsInteger[i] = AInt32SerializerDeserializer.getInt(bytes, offset + 1);
operandsFloating[i] = operandsInteger[i];
break;
case BIGINT:
- currentType = typeInt64;
+ currentType = ATypeTag.BIGINT;
operandsInteger[i] = AInt64SerializerDeserializer.getLong(bytes, offset + 1);
operandsFloating[i] = operandsInteger[i];
break;
case FLOAT:
- currentType = typeFloat;
+ currentType = ATypeTag.FLOAT;
operandsFloating[i] = AFloatSerializerDeserializer.getFloat(bytes, offset + 1);
break;
case DOUBLE:
- currentType = typeDouble;
+ currentType = ATypeTag.DOUBLE;
operandsFloating[i] = ADoubleSerializerDeserializer.getDouble(bytes, offset + 1);
break;
case DATE:
@@ -186,7 +206,7 @@
case DURATION:
case YEARMONTHDURATION:
case DAYTIMEDURATION:
- evaluateTemporalArthmeticOperation(typeTag);
+ evaluateTemporalArithmeticOperation(typeTag);
result.set(resultStorage);
return;
default:
@@ -201,17 +221,17 @@
ATypeTag.SERIALIZED_DAY_TIME_DURATION_TYPE_TAG);
}
- if (resultType < currentType) {
- resultType = currentType;
+ if (i == 0 || currentType.ordinal() > argTypeMax.ordinal()) {
+ argTypeMax = currentType;
}
}
+ ATypeTag resultType = getNumericResultType(argTypeMax);
+
long lres;
double dres;
switch (resultType) {
- case typeInt8:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT8);
+ case TINYINT:
lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
if (lres > Byte.MAX_VALUE) {
throw new OverflowException(getIdentifier());
@@ -220,11 +240,9 @@
throw new UnderflowException(getIdentifier());
}
aInt8.setValue((byte) lres);
- serde.serialize(aInt8, out);
+ int8Serde.serialize(aInt8, out);
break;
- case typeInt16:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT16);
+ case SMALLINT:
lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
if (lres > Short.MAX_VALUE) {
throw new OverflowException(getIdentifier());
@@ -233,11 +251,9 @@
throw new UnderflowException(getIdentifier());
}
aInt16.setValue((short) lres);
- serde.serialize(aInt16, out);
+ int16Serde.serialize(aInt16, out);
break;
- case typeInt32:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT32);
+ case INTEGER:
lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
if (lres > Integer.MAX_VALUE) {
throw new OverflowException(getIdentifier());
@@ -246,18 +262,14 @@
throw new UnderflowException(getIdentifier());
}
aInt32.setValue((int) lres);
- serde.serialize(aInt32, out);
+ int32Serde.serialize(aInt32, out);
break;
- case typeInt64:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AINT64);
+ case BIGINT:
lres = evaluateInteger(operandsInteger[0], operandsInteger[1]);
aInt64.setValue(lres);
- serde.serialize(aInt64, out);
+ int64Serde.serialize(aInt64, out);
break;
- case typeFloat:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.AFLOAT);
+ case FLOAT:
dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
if (dres > Float.MAX_VALUE) {
throw new OverflowException(getIdentifier());
@@ -266,20 +278,19 @@
throw new UnderflowException(getIdentifier());
}
aFloat.setValue((float) dres);
- serde.serialize(aFloat, out);
+ floatSerde.serialize(aFloat, out);
break;
- case typeDouble:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADOUBLE);
- aDouble.setValue(evaluateDouble(operandsFloating[0], operandsFloating[1]));
- serde.serialize(aDouble, out);
+ case DOUBLE:
+ dres = evaluateDouble(operandsFloating[0], operandsFloating[1]);
+ aDouble.setValue(dres);
+ doubleSerde.serialize(aDouble, out);
break;
}
result.set(resultStorage);
}
@SuppressWarnings("unchecked")
- private void evaluateTemporalArthmeticOperation(ATypeTag leftType) throws HyracksDataException {
+ private void evaluateTemporalArithmeticOperation(ATypeTag leftType) throws HyracksDataException {
byte[] bytes1 = argPtr1.getByteArray();
int offset1 = argPtr1.getStartOffset();
ATypeTag rightType = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
@@ -288,10 +299,7 @@
if (rightType == leftType) {
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADURATION);
-
- long leftChronon = 0, rightChronon = 0, dayTime = 0;
+ long leftChronon = 0, rightChronon = 0, dayTime;
int yearMonth = 0;
@@ -301,7 +309,6 @@
* GregorianCalendarSystem.CHRONON_OF_DAY;
rightChronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
* GregorianCalendarSystem.CHRONON_OF_DAY;
-
break;
case TIME:
leftChronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
@@ -329,20 +336,19 @@
dayTime = evaluateTimeInstanceArithmetic(leftChronon, rightChronon);
aDuration.setValue(yearMonth, dayTime);
-
- serde.serialize(aDuration, out);
+ durationSerde.serialize(aDuration, out);
} else {
long chronon = 0, dayTime = 0;
int yearMonth = 0;
ATypeTag resultType = null;
+ ISerializerDeserializer serde = null;
boolean isTimeOnly = false;
switch (leftType) {
case TIME:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ATIME);
+ serde = timeSerde;
chronon = ATimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
isTimeOnly = true;
resultType = ATypeTag.TIME;
@@ -362,15 +368,13 @@
}
break;
case DATE:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATE);
+ serde = dateSerde;
resultType = ATypeTag.DATE;
chronon = ADateSerializerDeserializer.getChronon(bytes0, offset0 + 1)
* GregorianCalendarSystem.CHRONON_OF_DAY;
case DATETIME:
if (leftType == ATypeTag.DATETIME) {
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATETIME);
+ serde = dateTimeSerde;
resultType = ATypeTag.DATETIME;
chronon = ADateTimeSerializerDeserializer.getChronon(bytes0, offset0 + 1);
}
@@ -398,14 +402,12 @@
AYearMonthDurationSerializerDeserializer.getYearMonth(bytes0, offset0 + 1);
switch (rightType) {
case DATETIME:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATETIME);
+ serde = dateTimeSerde;
resultType = ATypeTag.DATETIME;
chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
break;
case DATE:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATE);
+ serde = dateSerde;
resultType = ATypeTag.DATE;
chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
* GregorianCalendarSystem.CHRONON_OF_DAY;
@@ -425,22 +427,19 @@
}
switch (rightType) {
case DATETIME:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATETIME);
+ serde = dateTimeSerde;
resultType = ATypeTag.DATETIME;
chronon = ADateTimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
break;
case DATE:
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ADATE);
+ serde = dateSerde;
resultType = ATypeTag.DATE;
chronon = ADateSerializerDeserializer.getChronon(bytes1, offset1 + 1)
* GregorianCalendarSystem.CHRONON_OF_DAY;
break;
case TIME:
if (yearMonth == 0) {
- serde = SerializerDeserializerProvider.INSTANCE
- .getSerializerDeserializer(BuiltinType.ATIME);
+ serde = timeSerde;
resultType = ATypeTag.TIME;
chronon = ATimeSerializerDeserializer.getChronon(bytes1, offset1 + 1);
isTimeOnly = true;
@@ -486,4 +485,8 @@
}
};
}
+
+ protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+ return argTypeMax;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
deleted file mode 100644
index 0079a8a..0000000
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericCaretDescriptor.java
+++ /dev/null
@@ -1,80 +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.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.exceptions.OverflowException;
-import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-import com.google.common.math.LongMath;
-
-public class NumericCaretDescriptor extends AbstractNumericArithmeticEval {
-
- private static final long serialVersionUID = 1L;
-
- public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
- public IFunctionDescriptor createFunctionDescriptor() {
- return new NumericCaretDescriptor();
- }
- };
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
- */
- @Override
- protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
- if (rhs > Integer.MAX_VALUE) {
- throw new OverflowException(getIdentifier());
- }
- return LongMath.checkedPow(lhs, (int) rhs);
- }
-
- /* (non-Javadoc)
- * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
- */
- @Override
- protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
- return Math.pow(lhs, rhs);
- }
-
- /* (non-Javadoc)
- * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
- */
- @Override
- public FunctionIdentifier getIdentifier() {
- return BuiltinFunctions.CARET;
- }
-
- @Override
- protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
- throws HyracksDataException {
- throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
- }
-
- @Override
- protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
- throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
- }
-
-}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.java
new file mode 100644
index 0000000..dbb4737
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDegreesDescriptor.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.runtime.evaluators.functions;
+
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericDegreesDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NumericDegreesDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier()) {
+ @Override
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+ aDouble.setValue(Math.toDegrees(arg));
+ serialize(aDouble, doubleSerde, resultPointable);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NUMERIC_DEGREES;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
new file mode 100644
index 0000000..33edc7b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivDescriptor.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+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.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class NumericDivDescriptor extends AbstractNumericArithmeticEval {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NumericDivDescriptor();
+ }
+ };
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NUMERIC_DIV;
+ }
+
+ @Override
+ protected double evaluateDouble(double lhs, double rhs) {
+ return lhs / rhs;
+ }
+
+ @Override
+ protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+ if (rhs == 0) {
+ throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
+ }
+ if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
+ throw new OverflowException(getIdentifier());
+ }
+ return lhs / rhs;
+ }
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly) {
+ throw new NotImplementedException("Divide operation is not defined for temporal types");
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new UnsupportedTypeException(getIdentifier(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
index 77c94bf..868f3b5 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericDivideDescriptor.java
@@ -18,21 +18,18 @@
*/
package org.apache.asterix.runtime.evaluators.functions;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
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.exceptions.OverflowException;
import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class NumericDivideDescriptor extends AbstractNumericArithmeticEval {
-
private static final long serialVersionUID = 1L;
+
public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
public IFunctionDescriptor createFunctionDescriptor() {
return new NumericDivideDescriptor();
@@ -45,14 +42,8 @@
}
@Override
- protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
- if (rhs == 0) {
- throw new RuntimeDataException(ErrorCode.DIVISION_BY_ZERO);
- }
- if ((lhs == Long.MIN_VALUE) && (rhs == -1L)) {
- throw new OverflowException(getIdentifier());
- }
- return lhs / rhs;
+ protected ATypeTag getNumericResultType(ATypeTag argTypeMax) {
+ return argTypeMax.ordinal() < ATypeTag.FLOAT.ordinal() ? ATypeTag.DOUBLE : argTypeMax;
}
@Override
@@ -61,8 +52,12 @@
}
@Override
- protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
- throws HyracksDataException {
+ protected long evaluateInteger(long lhs, long rhs) {
+ throw new IllegalStateException();
+ }
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly) {
throw new NotImplementedException("Divide operation is not defined for temporal types");
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
new file mode 100644
index 0000000..ae7f019
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericPowerDescriptor.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.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.exceptions.OverflowException;
+import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.google.common.math.LongMath;
+
+public class NumericPowerDescriptor extends AbstractNumericArithmeticEval {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NumericPowerDescriptor();
+ }
+ };
+
+ /* (non-Javadoc)
+ * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateInteger(long, long)
+ */
+ @Override
+ protected long evaluateInteger(long lhs, long rhs) throws HyracksDataException {
+ if (rhs > Integer.MAX_VALUE) {
+ throw new OverflowException(getIdentifier());
+ }
+ return LongMath.checkedPow(lhs, (int) rhs);
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.asterix.runtime.evaluators.functions.AbstractNumericArithmeticEval#evaluateDouble(double, double)
+ */
+ @Override
+ protected double evaluateDouble(double lhs, double rhs) throws HyracksDataException {
+ return Math.pow(lhs, rhs);
+ }
+
+ /* (non-Javadoc)
+ * @see org.apache.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+ */
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NUMERIC_POWER;
+ }
+
+ @Override
+ protected long evaluateTimeDurationArithmetic(long chronon, int yearMonth, long dayTime, boolean isTimeOnly)
+ throws HyracksDataException {
+ throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_DURATION_TYPE_TAG);
+ }
+
+ @Override
+ protected long evaluateTimeInstanceArithmetic(long chronon0, long chronon1) throws HyracksDataException {
+ throw new UnsupportedTypeException(getIdentifier().getName(), ATypeTag.SERIALIZED_TIME_TYPE_TAG);
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.java
new file mode 100644
index 0000000..51352cb
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/NumericRadiansDescriptor.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.runtime.evaluators.functions;
+
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+
+public class NumericRadiansDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new NumericRadiansDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractUnaryNumericDoubleFunctionEval(ctx, args[0], getIdentifier()) {
+ @Override
+ protected void processDouble(double arg, IPointable resultPointable) throws HyracksDataException {
+ aDouble.setValue(Math.toRadians(arg));
+ serialize(aDouble, doubleSerde, resultPointable);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.NUMERIC_RADIANS;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
new file mode 100644
index 0000000..fb99438
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomDescriptor.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.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.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new RandomDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private final RandomHelper randomHelper = new RandomHelper(false);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+ randomHelper.nextDouble(result);
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.RANDOM;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
new file mode 100644
index 0000000..6dd5eac
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.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.functions.utils.RandomHelper;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class RandomWithSeedDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new RandomWithSeedDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(IHyracksTaskContext ctx) throws HyracksDataException {
+ return new IScalarEvaluator() {
+ private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+ private final IPointable arg0 = new VoidPointable();
+
+ private final RandomHelper randomHelper = new RandomHelper(true);
+
+ @Override
+ public void evaluate(IFrameTupleReference tuple, IPointable resultPointable)
+ throws HyracksDataException {
+ eval0.evaluate(tuple, arg0);
+
+ byte[] bytes = arg0.getByteArray();
+ int offset = arg0.getStartOffset();
+ ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[offset]];
+ switch (tt) {
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case FLOAT:
+ case DOUBLE:
+ randomHelper.setSeed(bytes, offset + 1, arg0.getLength() - 1);
+ randomHelper.nextDouble(resultPointable);
+ break;
+ default:
+ PointableHelper.setNull(resultPointable);
+ break;
+ }
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.RANDOM_WITH_SEED;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
new file mode 100644
index 0000000..0095291
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringReverseDescriptor.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.io.IOException;
+
+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.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
+
+public class StringReverseDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+ @Override
+ public IFunctionDescriptor createFunctionDescriptor() {
+ return new StringReverseDescriptor();
+ }
+ };
+
+ @Override
+ public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+ return new IScalarEvaluatorFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+ return new AbstractUnaryStringStringEval(ctx, args[0], getIdentifier()) {
+ @Override
+ void process(UTF8StringPointable inputString, IPointable resultPointable) throws IOException {
+ UTF8StringPointable.reverse(inputString, resultBuilder, resultArray);
+ resultPointable.set(resultArray.getByteArray(), 0, resultArray.getLength());
+ }
+ };
+ }
+ };
+ }
+
+ @Override
+ public FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.STRING_REVERSE;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
index 53f73ae..854eb8e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/Substring2Descriptor.java
@@ -86,8 +86,7 @@
byte[] bytes = argStart.getByteArray();
int offset = argStart.getStartOffset();
- int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset)
- - baseOffset;
+ int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 1, bytes, offset);
bytes = argString.getByteArray();
offset = argString.getStartOffset();
int len = argString.getLength();
@@ -98,7 +97,8 @@
string.set(bytes, offset + 1, len - 1);
array.reset();
try {
- UTF8StringPointable.substr(string, start, Integer.MAX_VALUE, builder, array);
+ int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+ UTF8StringPointable.substr(string, actualStart, Integer.MAX_VALUE, builder, array);
} catch (StringIndexOutOfBoundsException e) {
throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start);
} catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
index 148d0a4..526711c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SubstringDescriptor.java
@@ -93,8 +93,7 @@
byte[] bytes = argStart.getByteArray();
int offset = argStart.getStartOffset();
- int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset)
- - baseOffset;
+ int start = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes, offset);
bytes = argLen.getByteArray();
offset = argLen.getStartOffset();
@@ -110,7 +109,8 @@
string.set(bytes, offset + 1, length - 1);
array.reset();
try {
- UTF8StringPointable.substr(string, start, len, builder, array);
+ int actualStart = start >= 0 ? start - baseOffset : string.getStringLength() + start;
+ UTF8StringPointable.substr(string, actualStart, len, builder, array);
} catch (StringIndexOutOfBoundsException e) {
throw new RuntimeDataException(ErrorCode.OUT_OF_BOUND, getIdentifier(), 1, start + len - 1);
} catch (IOException e) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
new file mode 100644
index 0000000..7ed1ce5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.utils;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.security.SecureRandom;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.DataUtils;
+import org.apache.hyracks.data.std.util.GrowableArray;
+
+public final class RandomHelper {
+
+ private final SecureRandom random = new SecureRandom();
+
+ private final GrowableArray seed;
+
+ private final AMutableDouble aDouble = new AMutableDouble(0);
+
+ @SuppressWarnings("rawtypes")
+ private ISerializerDeserializer doubleSerde =
+ SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+
+ private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+ private final DataOutput dataOutput = resultStorage.getDataOutput();
+
+ public RandomHelper(boolean withSeed) {
+ seed = withSeed ? new GrowableArray(8) : null;
+ }
+
+ public void setSeed(byte[] bytes, int offset, int length) throws HyracksDataException {
+ if (seed == null) {
+ throw new IllegalStateException();
+ }
+
+ boolean sameSeed =
+ seed.getLength() == length && DataUtils.equalsInRange(seed.getByteArray(), 0, bytes, offset, length);
+
+ if (!sameSeed) {
+ try {
+ seed.reset();
+ seed.append(bytes, offset, length);
+ random.setSeed(seed.getByteArray());
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+ }
+
+ public void nextDouble(IPointable resultPointable) throws HyracksDataException {
+ aDouble.setValue(random.nextDouble());
+ resultStorage.reset();
+ doubleSerde.serialize(aDouble, dataOutput);
+ resultPointable.set(resultStorage);
+ }
+}
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 b05dbed..9fecf34 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
@@ -104,7 +104,12 @@
import org.apache.asterix.runtime.evaluators.comparisons.GreaterThanOrEqualsDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.LessThanDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.LessThanOrEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.MissingIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NanIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NegInfIfEqualsDescriptor;
import org.apache.asterix.runtime.evaluators.comparisons.NotEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.NullIfEqualsDescriptor;
+import org.apache.asterix.runtime.evaluators.comparisons.PosInfIfEqualsDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABinaryBase64StringConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABinaryHexStringConstructorDescriptor;
import org.apache.asterix.runtime.evaluators.constructors.ABooleanConstructorDescriptor;
@@ -180,9 +185,11 @@
import org.apache.asterix.runtime.evaluators.functions.NumericATanDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericAbsDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericAddDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.NumericCaretDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDivDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericPowerDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericCeilingDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericCosDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericDegreesDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericDivideDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericExpDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericFloorDescriptor;
@@ -190,6 +197,7 @@
import org.apache.asterix.runtime.evaluators.functions.NumericLogDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericModuloDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericMultiplyDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.NumericRadiansDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEven2Descriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericRoundHalfToEvenDescriptor;
@@ -201,6 +209,8 @@
import org.apache.asterix.runtime.evaluators.functions.NumericTruncDescriptor;
import org.apache.asterix.runtime.evaluators.functions.NumericUnaryMinusDescriptor;
import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
@@ -230,6 +240,7 @@
import org.apache.asterix.runtime.evaluators.functions.StringRepeatDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringReplaceDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringReplaceWithLimitDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.StringReverseDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringSplitDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringStartsWithDescriptor;
import org.apache.asterix.runtime.evaluators.functions.StringToCodePointDescriptor;
@@ -460,6 +471,7 @@
fc.add(CreateUUIDDescriptor.FACTORY);
fc.add(UUIDDescriptor.FACTORY);
fc.add(CreateQueryUIDDescriptor.FACTORY);
+ fc.add(RandomDescriptor.FACTORY);
fc.add(CurrentDateDescriptor.FACTORY);
fc.add(CurrentTimeDescriptor.FACTORY);
fc.add(CurrentDateTimeDescriptor.FACTORY);
@@ -481,10 +493,11 @@
fc.addGenerated(NumericUnaryMinusDescriptor.FACTORY);
fc.addGenerated(NumericAddDescriptor.FACTORY);
fc.addGenerated(NumericDivideDescriptor.FACTORY);
+ fc.addGenerated(NumericDivDescriptor.FACTORY);
fc.addGenerated(NumericMultiplyDescriptor.FACTORY);
fc.addGenerated(NumericSubDescriptor.FACTORY);
fc.addGenerated(NumericModuloDescriptor.FACTORY);
- fc.addGenerated(NumericCaretDescriptor.FACTORY);
+ fc.addGenerated(NumericPowerDescriptor.FACTORY);
fc.addGenerated(NotDescriptor.FACTORY);
fc.addGenerated(LenDescriptor.FACTORY);
fc.addGenerated(NumericAbsDescriptor.FACTORY);
@@ -496,6 +509,8 @@
fc.addGenerated(NumericACosDescriptor.FACTORY);
fc.addGenerated(NumericASinDescriptor.FACTORY);
fc.addGenerated(NumericATanDescriptor.FACTORY);
+ fc.addGenerated(NumericDegreesDescriptor.FACTORY);
+ fc.addGenerated(NumericRadiansDescriptor.FACTORY);
fc.addGenerated(NumericCosDescriptor.FACTORY);
fc.addGenerated(NumericSinDescriptor.FACTORY);
fc.addGenerated(NumericTanDescriptor.FACTORY);
@@ -515,6 +530,13 @@
fc.addGenerated(LessThanOrEqualsDescriptor.FACTORY);
fc.addGenerated(NotEqualsDescriptor.FACTORY);
+ // If-Equals functions
+ fc.addGenerated(MissingIfEqualsDescriptor.FACTORY);
+ fc.addGenerated(NullIfEqualsDescriptor.FACTORY);
+ fc.addGenerated(NanIfEqualsDescriptor.FACTORY);
+ fc.addGenerated(PosInfIfEqualsDescriptor.FACTORY);
+ fc.addGenerated(NegInfIfEqualsDescriptor.FACTORY);
+
// Binary functions
fc.addGenerated(BinaryLengthDescriptor.FACTORY);
fc.addGenerated(ParseBinaryDescriptor.FACTORY);
@@ -561,6 +583,7 @@
fc.addGenerated(StringRepeatDescriptor.FACTORY);
fc.addGenerated(StringReplaceDescriptor.FACTORY);
fc.addGenerated(StringReplaceWithLimitDescriptor.FACTORY);
+ fc.addGenerated(StringReverseDescriptor.FACTORY);
fc.addGenerated(StringSplitDescriptor.FACTORY);
// Constructors
@@ -714,6 +737,9 @@
// Record function
fc.addGenerated(RecordPairsDescriptor.FACTORY);
+ // Other functions
+ fc.addGenerated(RandomWithSeedDescriptor.FACTORY);
+
ServiceLoader.load(IFunctionRegistrant.class).iterator().forEachRemaining(c -> c.register(fc));
return fc;
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
index 2415556..4130490 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMIndexBulkLoadOperatorNodePushable.java
@@ -18,10 +18,13 @@
*/
package org.apache.asterix.runtime.operators;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.runtime.operators.LSMIndexBulkLoadOperatorDescriptor.BulkLoadUsage;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -33,7 +36,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
-import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
public class LSMIndexBulkLoadOperatorNodePushable extends IndexBulkLoadOperatorNodePushable {
@@ -71,29 +73,26 @@
@Override
protected void initializeBulkLoader() throws HyracksDataException {
ILSMIndex targetIndex = (ILSMIndex) index;
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, LSMComponentId.DEFAULT_COMPONENT_ID);
if (usage.equals(BulkLoadUsage.LOAD)) {
- // for a loaded dataset, we use the default Id 0 which is guaranteed to be smaller
- // than Ids of all memory components
-
- // TODO handle component Id for datasets loaded multiple times
- // TODO move this piece of code to io operation callback
- bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
- ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
- LSMComponentIdUtils.persist(LSMComponentId.DEFAULT_COMPONENT_ID, diskComponent.getMetadata());
+ bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ parameters);
} else {
primaryIndexHelper.open();
primaryIndex = (ILSMIndex) primaryIndexHelper.getIndexInstance();
List<ILSMDiskComponent> primaryComponents = primaryIndex.getDiskComponents();
- bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
if (!primaryComponents.isEmpty()) {
- // TODO move this piece of code to io operation callback
- // Ideally, this should be done in io operation callback when a bulk load operation is finished
- // However, currently we don't have an extensible callback mechanism to support this
ILSMComponentId bulkloadId = LSMComponentIdUtils.union(primaryComponents.get(0).getId(),
primaryComponents.get(primaryComponents.size() - 1).getId());
- ILSMDiskComponent diskComponent = ((LSMIndexDiskComponentBulkLoader) bulkLoader).getComponent();
- LSMComponentIdUtils.persist(bulkloadId, diskComponent.getMetadata());
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, bulkloadId);
+ } else {
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID,
+ LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
}
+ bulkLoader = targetIndex.createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ parameters);
+
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
index 6d9ec47..1029d6f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexBulkLoadNodePushable.java
@@ -19,7 +19,10 @@
package org.apache.asterix.runtime.operators;
import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.asterix.common.ioopcallbacks.LSMIOOperationCallback;
import org.apache.asterix.runtime.operators.LSMSecondaryIndexCreationTupleProcessorNodePushable.DeletedTupleCounter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
@@ -32,9 +35,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexDiskComponentBulkLoader;
-import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
/**
+ * Note: only used with correlated merge policy
* This operator node is used to bulk load incoming tuples (scanned from the primary index)
* into multiple disk components of the secondary index.
* Incoming tuple format:
@@ -182,14 +185,12 @@
private void loadNewComponent(int componentPos) throws HyracksDataException {
endCurrentComponent();
-
int numTuples = getNumDeletedTuples(componentPos);
ILSMDiskComponent primaryComponent = primaryIndex.getDiskComponents().get(componentPos);
- componentBulkLoader =
- (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples, false);
- ILSMDiskComponent diskComponent = componentBulkLoader.getComponent();
- // TODO move this piece of code to io operation callback
- LSMComponentIdUtils.persist(primaryComponent.getId(), diskComponent.getMetadata());
+ Map<String, Object> parameters = new HashMap<>();
+ parameters.put(LSMIOOperationCallback.KEY_FLUSHED_COMPONENT_ID, primaryComponent.getId());
+ componentBulkLoader = (LSMIndexDiskComponentBulkLoader) secondaryIndex.createBulkLoader(1.0f, false, numTuples,
+ false, parameters);
}
private void addAntiMatterTuple(ITupleReference tuple) throws HyracksDataException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
index 9376d1b..ac7fc89 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryIndexCreationTupleProcessorNodePushable.java
@@ -276,7 +276,7 @@
private boolean equalPrimaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
for (int i = numTagFields + numSecondaryKeys; i < numTagFields + numPrimaryKeys + numSecondaryKeys; i++) {
- if (!equalField(tuple1, tuple2, i)) {
+ if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
return false;
}
}
@@ -285,16 +285,10 @@
private boolean equalSecondaryKeys(ITupleReference tuple1, ITupleReference tuple2) {
for (int i = numTagFields; i < numTagFields + numSecondaryKeys; i++) {
- if (!equalField(tuple1, tuple2, i)) {
+ if (!TupleUtils.equalFields(tuple1, tuple2, i)) {
return false;
}
}
return true;
}
-
- private boolean equalField(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
- return LSMSecondaryUpsertOperatorNodePushable.equals(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx),
- tuple1.getFieldLength(fIdx), tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx),
- tuple2.getFieldLength(fIdx));
- }
}
\ No newline at end of file
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 a22e5e7..b928131 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
@@ -28,6 +28,7 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -55,7 +56,7 @@
public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
- private int numberOfFields;
+ private final int numberOfFields;
private AbstractIndexModificationOperationCallback abstractModCallback;
public LSMSecondaryUpsertOperatorNodePushable(IHyracksTaskContext ctx, int partition,
@@ -74,31 +75,6 @@
abstractModCallback = (AbstractIndexModificationOperationCallback) modCallback;
}
- public static boolean equals(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
- if (aLength != bLength) {
- return false;
- }
- for (int i = 0; i < aLength; i++) {
- if (a[aOffset + i] != b[bOffset + i]) {
- return false;
- }
- }
- return true;
- }
-
- public static boolean equalTuples(PermutingFrameTupleReference t1, PermutingFrameTupleReference t2, int numOfFields)
- throws HyracksDataException {
- byte[] t1Data = t1.getFieldData(0);
- byte[] t2Data = t2.getFieldData(0);
- for (int i = 0; i < numOfFields; i++) {
- if (!equals(t1Data, t1.getFieldStart(i), t1.getFieldLength(i), t2Data, t2.getFieldStart(i),
- t2.getFieldLength(i))) {
- return false;
- }
- }
- return true;
- }
-
@Override
public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
accessor.reset(buffer);
@@ -117,7 +93,7 @@
}
// At least, one is not null
// If they are equal, then we skip
- if (equalTuples(tuple, prevValueTuple, numberOfFields)) {
+ if (TupleUtils.equalTuples(tuple, prevValueTuple, numberOfFields)) {
continue;
}
if (!isOldValueMissing) {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
index d61e9a0..7e42d14 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/std/FlushDatasetOperatorDescriptor.java
@@ -22,6 +22,7 @@
import org.apache.asterix.common.api.IDatasetLifecycleManager;
import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.context.DatasetInfo;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.transactions.DatasetId;
import org.apache.asterix.common.transactions.ILockManager;
@@ -82,7 +83,13 @@
// lock the dataset granule
lockManager.lock(datasetId, -1, LockMode.S, txnCtx);
// flush the dataset synchronously
- datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+ DatasetInfo datasetInfo = datasetLifeCycleManager.getDatasetInfo(datasetId.getId());
+ // TODO: Remove the isOpen check and let it fail if flush is requested for a dataset that is closed
+ synchronized (datasetLifeCycleManager) {
+ if (datasetInfo.isOpen()) {
+ datasetLifeCycleManager.flushDataset(datasetId.getId(), false);
+ }
+ }
} catch (ACIDException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index fd6dd79..b372dbd 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -372,6 +372,7 @@
<usedDependency>org.apache.asterix:asterix-external-data</usedDependency>
<usedDependency>org.codehaus.mojo.appassembler:appassembler-booter</usedDependency>
<usedDependency>org.apache.asterix:asterix-fuzzyjoin</usedDependency>
+ <usedDependency>org.apache.asterix:asterix-geo</usedDependency>
</usedDependencies>
</configuration>
</plugin>
@@ -625,5 +626,14 @@
<artifactId>hyracks-test-support</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>org.apache.asterix</groupId>
+ <artifactId>asterix-geo</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
index f7e0e34..4f3caed 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/start-sample-cluster.sh
@@ -39,13 +39,56 @@
export JAVA_HOME
fi
-[ -z "$JAVA_HOME" ] && {
- echo "JAVA_HOME not set"
- exit 1
-}
-"$JAVA_HOME/bin/java" -version 2>&1 | grep -q '1\.[89]' || {
+# OS specific support. $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+ CYGWIN*) cygwin=true ;;
+ Darwin*) darwin=true
+ if [ -z "$JAVA_VERSION" ] ; then
+ JAVA_VERSION="CurrentJDK"
+ else
+ echo "Using Java version: $JAVA_VERSION"
+ fi
+ if [ -z "$JAVA_HOME" ]; then
+ if [ -x "/usr/libexec/java_home" ]; then
+ JAVA_HOME=`/usr/libexec/java_home`
+ else
+ JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+ fi
+ fi
+ ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+ if [ -r /etc/gentoo-release ] ; then
+ JAVA_HOME=`java-config --jre-home`
+ fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+ [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+ [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+ if [ -n "$JAVA_HOME" ] ; then
+ if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+ # IBM's JDK on AIX uses strange locations for the executables
+ JAVACMD="$JAVA_HOME/jre/sh/java"
+ else
+ JAVACMD="$JAVA_HOME/bin/java"
+ fi
+ else
+ JAVACMD=`which java`
+ fi
+fi
+
+"$JAVACMD" -version 2>&1 | grep -q '1\.[89]' || {
echo "JAVA_HOME must be at version 1.8 or later:"
- "$JAVA_HOME/bin/java" -version
+ "$JAVACMD" -version
exit 2
}
DIRNAME=$(dirname "$0")
diff --git a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
index dbce377..522fb7c 100755
--- a/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
+++ b/asterixdb/asterix-server/src/main/opt/local/bin/stop-sample-cluster.sh
@@ -39,13 +39,69 @@
export JAVA_HOME
fi
-[ -z "$JAVA_HOME" ] && {
- echo "JAVA_HOME not set"
- exit 1
-}
-"$JAVA_HOME/bin/java" -version 2>&1 | grep -q '1\.[89]' || {
+while [ -n "$1" ]; do
+ case $1 in
+ -f|-force) force=1;;
+ -help|--help|-usage|--usage) usage; exit 0;;
+ *) echo "ERROR: unknown argument '$1'"; usage; exit 1;;
+ esac
+ shift
+done
+
+if [ -z "$JAVA_HOME" -a -x /usr/libexec/java_home ]; then
+ JAVA_HOME=$(/usr/libexec/java_home)
+ export JAVA_HOME
+fi
+
+# OS specific support. $var _must_ be set to either true or false.
+cygwin=false;
+darwin=false;
+case "`uname`" in
+ CYGWIN*) cygwin=true ;;
+ Darwin*) darwin=true
+ if [ -z "$JAVA_VERSION" ] ; then
+ JAVA_VERSION="CurrentJDK"
+ else
+ echo "Using Java version: $JAVA_VERSION"
+ fi
+ if [ -z "$JAVA_HOME" ]; then
+ if [ -x "/usr/libexec/java_home" ]; then
+ JAVA_HOME=`/usr/libexec/java_home`
+ else
+ JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Versions/${JAVA_VERSION}/Home
+ fi
+ fi
+ ;;
+esac
+
+if [ -z "$JAVA_HOME" ] ; then
+ if [ -r /etc/gentoo-release ] ; then
+ JAVA_HOME=`java-config --jre-home`
+ fi
+fi
+
+# For Cygwin, ensure paths are in UNIX format before anything is touched
+if $cygwin ; then
+ [ -n "$JAVA_HOME" ] && JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
+ [ -n "$CLASSPATH" ] && CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
+fi
+
+# If a specific java binary isn't specified search for the standard 'java' binary
+if [ -z "$JAVACMD" ] ; then
+ if [ -n "$JAVA_HOME" ] ; then
+ if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+ # IBM's JDK on AIX uses strange locations for the executables
+ JAVACMD="$JAVA_HOME/jre/sh/java"
+ else
+ JAVACMD="$JAVA_HOME/bin/java"
+ fi
+ else
+ JAVACMD=`which java`
+ fi
+fi
+"$JAVACMD" -version 2>&1 | grep -q '1\.[89]' || {
echo "JAVA_HOME must be at version 1.8 or later:"
- "$JAVA_HOME/bin/java" -version
+ "$JAVACMD" -version
exit 2
}
DIRNAME=$(dirname "$0")
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
index 766402c..defd378 100644
--- a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/SampleLocalClusterIT.java
@@ -23,11 +23,8 @@
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
-import java.io.StringWriter;
-import java.net.URI;
import java.net.URL;
import java.util.ArrayList;
-import java.util.Collections;
import java.util.List;
import org.apache.asterix.common.utils.Servlets;
@@ -36,7 +33,6 @@
import org.apache.asterix.test.common.TestHelper;
import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.IOUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
@@ -46,6 +42,10 @@
import org.junit.rules.TestRule;
import org.junit.runners.MethodSorters;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class SampleLocalClusterIT {
@@ -118,11 +118,13 @@
@Test
public void test1_sanityQuery() throws Exception {
TestExecutor testExecutor = new TestExecutor();
- InputStream resultStream = testExecutor.executeQuery("1+1", OutputFormat.ADM,
- new URI("http", null, "127.0.0.1", 19002, Servlets.AQL_QUERY, null, null), Collections.emptyList());
- StringWriter sw = new StringWriter();
- IOUtils.copy(resultStream, sw);
- Assert.assertEquals("2", sw.toString().trim());
+ InputStream resultStream = testExecutor.executeQueryService("1+1;",
+ testExecutor.getEndpoint(Servlets.QUERY_SERVICE), OutputFormat.ADM);
+ final ObjectMapper objectMapper = new ObjectMapper();
+ final ObjectNode response = objectMapper.readValue(resultStream, ObjectNode.class);
+ final JsonNode result = response.get("results");
+ Assert.assertEquals(1, result.size());
+ Assert.assertEquals(2, result.get(0).asInt());
}
@Test
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 07dcf5d..e6e522e5 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -163,5 +163,9 @@
<groupId>commons-codec</groupId>
<artifactId>commons-codec</artifactId>
</dependency>
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 0375c30..e1963cb 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -186,7 +186,7 @@
}
resourceCache.put(resource.getPath(), resource);
- indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(0);
+ indexCheckpointManagerProvider.get(DatasetResourceReference.of(resource)).init(null, 0);
//if replication enabled, send resource metadata info to remote nodes
if (isReplicationEnabled) {
createReplicationJob(ReplicationOperation.REPLICATE, resourceFile);
@@ -429,15 +429,20 @@
}
private void deleteIndexInvalidComponents(File index) throws IOException, ParseException {
+ final Format formatter = THREAD_LOCAL_FORMATTER.get();
+ final File[] indexComponentFiles = index.listFiles(COMPONENT_FILES_FILTER);
+ if (indexComponentFiles == null) {
+ throw new IOException(index + " doesn't exist or an IO error occurred");
+ }
final Optional<String> validComponentTimestamp = getIndexCheckpointManager(index).getValidComponentTimestamp();
if (!validComponentTimestamp.isPresent()) {
- // index doesn't have any components
- return;
- }
- final Format formatter = THREAD_LOCAL_FORMATTER.get();
- final Date validTimestamp = (Date) formatter.parseObject(validComponentTimestamp.get());
- final File[] indexComponentFiles = index.listFiles(COMPONENT_FILES_FILTER);
- if (indexComponentFiles != null) {
+ // index doesn't have any valid component, delete all
+ for (File componentFile : indexComponentFiles) {
+ LOGGER.info(() -> "Deleting invalid component file: " + componentFile.getAbsolutePath());
+ Files.delete(componentFile.toPath());
+ }
+ } else {
+ final Date validTimestamp = (Date) formatter.parseObject(validComponentTimestamp.get());
for (File componentFile : indexComponentFiles) {
// delete any file with startTime > validTimestamp
final String fileStartTimeStr =
@@ -505,7 +510,8 @@
* e.g. a component file 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439_b
* will return a component id 2018-01-08-01-08-50-439_2018-01-08-01-08-50-439
*
- * @param componentFile any component file
+ * @param componentFile
+ * any component file
* @return The component id
*/
public static String getComponentId(String componentFile) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index c91d233..b26342e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -22,7 +22,6 @@
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
-import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import org.apache.asterix.common.exceptions.ACIDException;
@@ -36,30 +35,37 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMaps;
+import it.unimi.dsi.fastutil.longs.Long2LongOpenHashMap;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
/**
* A concurrent implementation of the ILockManager interface.
*
* @see ResourceGroupTable
* @see ResourceGroup
*/
+@SuppressWarnings("squid:RedundantThrowsDeclarationCheck") // throws ACIDException
public class ConcurrentLockManager implements ILockManager, ILifeCycleComponent {
static final Logger LOGGER = LogManager.getLogger();
static final Level LVL = Level.TRACE;
- public static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
+ private static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
- public static final int NIL = -1;
- public static final long NILL = -1L;
+ private static final int NIL = -1;
+ private static final long NILL = -1L;
- public static final boolean DEBUG_MODE = false;//true
- public static final boolean CHECK_CONSISTENCY = false;
+ private static final boolean DEBUG_MODE = false;//true
+ private static final boolean CHECK_CONSISTENCY = false;
- private ResourceGroupTable table;
- private ResourceArenaManager resArenaMgr;
- private RequestArenaManager reqArenaMgr;
- private JobArenaManager jobArenaMgr;
- private ConcurrentHashMap<Long, Long> txnId2TxnSlotMap;
- private LockManagerStats stats = new LockManagerStats(10000);
+ private final ResourceGroupTable table;
+ private final ResourceArenaManager resArenaMgr;
+ private final RequestArenaManager reqArenaMgr;
+ private final JobArenaManager jobArenaMgr;
+ private final Long2LongMap txnId2TxnSlotMap;
+ private final LockManagerStats stats = new LockManagerStats(10000);
enum LockAction {
ERR(false, false),
@@ -77,7 +83,7 @@
}
}
- static LockAction[][] ACTION_MATRIX = {
+ private static final LockAction[][] ACTION_MATRIX = {
// new NL IS IX S X
{ LockAction.ERR, LockAction.UPD, LockAction.UPD, LockAction.UPD, LockAction.UPD }, // NL
{ LockAction.ERR, LockAction.GET, LockAction.UPD, LockAction.UPD, LockAction.WAIT }, // IS
@@ -97,7 +103,7 @@
resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
- txnId2TxnSlotMap = new ConcurrentHashMap<>();
+ txnId2TxnSlotMap = Long2LongMaps.synchronize(new Long2LongOpenHashMap());
}
@Override
@@ -129,10 +135,10 @@
enqueueWaiter(group, reqSlot, resSlot, jobSlot, act, txnContext);
break;
}
- //no break
+ //fall-through
case UPD:
resArenaMgr.setMaxMode(resSlot, lockMode);
- // no break
+ //fall-through
case GET:
addHolder(reqSlot, resSlot, jobSlot);
locked = true;
@@ -185,31 +191,35 @@
void pop();
}
- static class NOPTracker implements DeadlockTracker {
+ private static class NOPTracker implements DeadlockTracker {
static final DeadlockTracker INSTANCE = new NOPTracker();
@Override
public void pushResource(long resSlot) {
+ // no-op
}
@Override
public void pushRequest(long reqSlot) {
+ // no-op
}
@Override
public void pushJob(long jobSlot) {
+ // no-op
}
@Override
public void pop() {
+ // no-op
}
}
- static class CollectingTracker implements DeadlockTracker {
+ private static class CollectingTracker implements DeadlockTracker {
static final boolean DEBUG = false;
- ArrayList<Long> slots = new ArrayList<>();
+ LongList slots = new LongArrayList();
ArrayList<String> types = new ArrayList<>();
@Override
@@ -217,7 +227,7 @@
types.add("Resource");
slots.add(resSlot);
if (DEBUG) {
- System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+ LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
}
}
@@ -226,7 +236,7 @@
types.add("Request");
slots.add(reqSlot);
if (DEBUG) {
- System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+ LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
}
}
@@ -235,24 +245,24 @@
types.add("Job");
slots.add(jobSlot);
if (DEBUG) {
- System.err.println("push " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+ LOGGER.info("push " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
}
}
@Override
public void pop() {
if (DEBUG) {
- System.err.println("pop " + types.get(types.size() - 1) + " " + slots.get(slots.size() - 1));
+ LOGGER.info("pop " + types.get(types.size() - 1) + " " + slots.getLong(slots.size() - 1));
}
types.remove(types.size() - 1);
- slots.remove(slots.size() - 1);
+ slots.removeLong(slots.size() - 1);
}
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
for (int i = 0; i < slots.size(); ++i) {
- sb.append(types.get(i) + " " + TypeUtil.Global.toString(slots.get(i)) + "\n");
+ sb.append(types.get(i)).append(" ").append(TypeUtil.Global.toString(slots.getLong(i))).append("\n");
}
return sb.toString();
}
@@ -270,15 +280,11 @@
* @return true if a cycle would be introduced, false otherwise
*/
private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
- if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
- /**
- * Due to the deadlock-free locking protocol, deadlock is not possible.
- * So, this method always returns false.
- */
- return false;
- } else {
- return introducesDeadlock(resSlot, jobSlot, tracker, 0);
- }
+ /*
+ * Due to the deadlock-free locking protocol, deadlock is not possible.
+ * So, this method always returns false in that case
+ */
+ return !ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL && introducesDeadlock(resSlot, jobSlot, tracker, 0);
}
private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker,
@@ -298,20 +304,20 @@
// The scanWaiters flag indicates if we are currently scanning the waiters (true) or the upgraders
// (false).
boolean scanWaiters = true;
- long waiter = jobArenaMgr.getLastWaiter(holderJobSlot);
- if (waiter < 0 && scanWaiters) {
+ long jobWaiter = jobArenaMgr.getLastWaiter(holderJobSlot);
+ if (jobWaiter < 0) {
scanWaiters = false;
- waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+ jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
}
- while (waiter >= 0) {
- long waitingOnResSlot = reqArenaMgr.getResourceId(waiter);
+ while (jobWaiter >= 0) {
+ long waitingOnResSlot = reqArenaMgr.getResourceId(jobWaiter);
if (introducesDeadlock(waitingOnResSlot, jobSlot, tracker, depth + 1)) {
return true;
}
- waiter = reqArenaMgr.getNextJobRequest(waiter);
- if (waiter < 0 && scanWaiters) {
+ jobWaiter = reqArenaMgr.getNextJobRequest(jobWaiter);
+ if (jobWaiter < 0 && scanWaiters) {
scanWaiters = false;
- waiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
+ jobWaiter = jobArenaMgr.getLastUpgrader(holderJobSlot);
}
}
@@ -407,7 +413,7 @@
switch (act) {
case UPD:
resArenaMgr.setMaxMode(resSlot, lockMode);
- // no break
+ //fall-through
case GET:
addHolder(reqSlot, resSlot, jobSlot);
return true;
@@ -532,8 +538,8 @@
stats.releaseLocks();
long txnId = txnContext.getTxnId().getId();
- Long jobSlot = txnId2TxnSlotMap.get(txnId);
- if (jobSlot == null) {
+ long jobSlot = txnId2TxnSlotMap.get(txnId);
+ if (jobSlot == 0) {
// we don't know the job, so there are no locks for it - we're done
return;
}
@@ -565,15 +571,15 @@
}
private long findOrAllocJobSlot(long txnId) {
- Long jobSlot = txnId2TxnSlotMap.get(txnId);
- if (jobSlot == null) {
- jobSlot = new Long(jobArenaMgr.allocate());
+ long jobSlot = txnId2TxnSlotMap.get(txnId);
+ if (jobSlot == 0) {
+ jobSlot = jobArenaMgr.allocate();
if (DEBUG_MODE) {
LOGGER.trace("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + txnId + ")");
}
jobArenaMgr.setTxnId(jobSlot, txnId);
- Long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
- if (oldSlot != null) {
+ long oldSlot = txnId2TxnSlotMap.putIfAbsent(txnId, jobSlot);
+ if (oldSlot != 0) {
// if another thread allocated a slot for this jobThreadId between
// get(..) and putIfAbsent(..), we'll use that slot and
// deallocate the one we allocated
@@ -584,7 +590,7 @@
jobSlot = oldSlot;
}
}
- assert (jobSlot >= 0);
+ assert jobSlot > 0;
return jobSlot;
}
@@ -754,7 +760,7 @@
void remove(long request, long resource, long job);
}
- final Queue waiter = new Queue() {
+ private final Queue waiter = new Queue() {
@Override
public void add(long request, long resource, long job) {
long waiter = resArenaMgr.getFirstWaiter(resource);
@@ -788,7 +794,7 @@
}
};
- final Queue upgrader = new Queue() {
+ private final Queue upgrader = new Queue() {
@Override
public void add(long request, long resource, long job) {
long upgrader = resArenaMgr.getFirstUpgrader(resource);
@@ -987,6 +993,7 @@
}
}
} catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
throw new IllegalStateException("interrupted", e);
}
}
@@ -1015,8 +1022,8 @@
* @return the slot of the request, if the lock request is found, NILL otherwise
*/
private long findLockInJobQueue(final int dsId, final int entityHashValue, final long txnId, byte lockMode) {
- Long jobSlot = txnId2TxnSlotMap.get(txnId);
- if (jobSlot == null) {
+ long jobSlot = txnId2TxnSlotMap.get(txnId);
+ if (jobSlot == 0) {
return NILL;
}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
index 26261c2..6a5372e 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
@@ -19,17 +19,17 @@
package org.apache.asterix.transaction.management.service.locking;
-import java.util.concurrent.ConcurrentHashMap;
+import it.unimi.dsi.fastutil.longs.Long2LongMap;
public class DumpTablePrinter implements TablePrinter {
private ResourceGroupTable table;
private ResourceArenaManager resArenaMgr;
private RequestArenaManager reqArenaMgr;
private JobArenaManager jobArenaMgr;
- private ConcurrentHashMap<Long, Long> txnIdToJobSlotMap;
+ private Long2LongMap txnIdToJobSlotMap;
DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
- JobArenaManager jobArenaMgr, ConcurrentHashMap<Long, Long> txnIdToJobSlotMap) {
+ JobArenaManager jobArenaMgr, Long2LongMap txnIdToJobSlotMap) {
this.table = table;
this.resArenaMgr = resArenaMgr;
this.reqArenaMgr = reqArenaMgr;
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
deleted file mode 100644
index 5bf5ad6..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
+++ /dev/null
@@ -1,595 +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.transaction.management.service.locking;
-
-import java.util.ArrayList;
-
-/**
- * PrimitiveIntHashMap supports primitive int type as key and value.
- * The hash map grows when the available slots in a bucket are overflowed.
- * Also, the hash map shrinks according to the following shrink policy.
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class PrimitiveIntHashMap {
- private final int CHILD_BUCKETS; //INIT_NUM_OF_BUCKETS;
- private final int NUM_OF_SLOTS; //NUM_OF_SLOTS_IN_A_BUCKET;
- private final int SHRINK_TIMER_THRESHOLD;
-
- private int occupiedSlots;
- private ArrayList<ChildIntArrayManager> pArray; //parent array
- private int hashMod;
- private long shrinkTimer;
- private boolean isShrinkTimerOn;
- private int iterBucketIndex;
- private int iterSlotIndex;
- private int iterChildIndex;
- private KeyValuePair iterPair;
-
- // ////////////////////////////////////////////////
- // // begin of unit test
- // ////////////////////////////////////////////////
- //
- // /**
- // * @param args
- // */
- // public static void main(String[] args) {
- // int i, j;
- // int k = 0;
- // int num = 5;
- // int key[] = new int[500];
- // int val[] = new int[500];
- // KeyValuePair pair;
- // PrimitiveIntHashMap map = new PrimitiveIntHashMap(1<<4, 1<<3, 5);
- //
- // for (j=0; j < num; j++) {
- //
- // k += 100;
- // //generate data
- // for (i=0; i < k; i++) {
- // key[i] = i;
- // val[i] = i;
- // }
- //
- // //put data to map
- // for (i=0; i < k-30; i++) {
- // map.put(key[i], val[i]);
- // }
- //
- // //put data to map
- // for (i=0; i < k-30; i++) {
- // map.put(key[i], val[i]);
- // }
- //
- // map.beginIterate();
- // pair = map.getNextKeyValue();
- // i = 0;
- // while (pair != null) {
- // i++;
- // System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
- // pair = map.getNextKeyValue();
- // }
- //
- // //System.out.println(map.prettyPrint());
- //
- // for (i=k-20; i< k; i++) { //skip X70~X79
- // map.put(key[i], val[i]);
- // }
- //
- // System.out.println(map.prettyPrint());
- //
- // //remove data to map
- // for (i=0; i < k-10; i++) {
- // map.remove(key[i]);
- // try {
- // Thread.currentThread().sleep(1);
- // } catch (InterruptedException e) {
- // e.printStackTrace();
- // }
- // }
- //
- // map.beginIterate();
- // pair = map.getNextKeyValue();
- // i = 0;
- // while (pair != null) {
- // i++;
- // System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
- // pair = map.getNextKeyValue();
- // }
- //
- // //remove data to map
- // for (i=0; i < k-10; i++) {
- // map.remove(key[i]);
- // try {
- // Thread.currentThread().sleep(1);
- // } catch (InterruptedException e) {
- // // TODO Auto-generated catch block
- // e.printStackTrace();
- // }
- // }
- //
- // System.out.println(map.prettyPrint());
- //
- // //get data from map
- // for (i=0; i < k; i++) {
- // System.out.println(""+i+"=> key:"+ key[i] + ", val:"+val[i] +", result: " + map.get(key[i]));
- // }
- // }
- //
- // map.beginIterate();
- // pair = map.getNextKeyValue();
- // i = 0;
- // while (pair != null) {
- // i++;
- // System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
- // pair = map.getNextKeyValue();
- // }
- // }
- //
- // ////////////////////////////////////////////////
- // // end of unit test
- // ////////////////////////////////////////////////
-
- public PrimitiveIntHashMap() {
- CHILD_BUCKETS = 1 << 9; //INIT_NUM_OF_BUCKETS;
- NUM_OF_SLOTS = 1 << 3; //NUM_OF_SLOTS_IN_A_BUCKET;
- SHRINK_TIMER_THRESHOLD = 120000; //2min
- pArray = new ArrayList<ChildIntArrayManager>();
- pArray.add(new ChildIntArrayManager(this));
- hashMod = CHILD_BUCKETS;
- occupiedSlots = 0;
- iterPair = new KeyValuePair();
- }
-
- public PrimitiveIntHashMap(int childBuckets, int numOfSlots, int shrinkTimerThreshold) {
- CHILD_BUCKETS = childBuckets;
- NUM_OF_SLOTS = numOfSlots;
- SHRINK_TIMER_THRESHOLD = shrinkTimerThreshold;
- pArray = new ArrayList<ChildIntArrayManager>();
- pArray.add(new ChildIntArrayManager(this));
- hashMod = CHILD_BUCKETS;
- occupiedSlots = 0;
- iterPair = new KeyValuePair();
- }
-
- public void put(int key, int value) {
- int growCount = 0;
- int bucketNum = hash(key);
- ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
- while (child.isFull(bucketNum % CHILD_BUCKETS)) {
- growHashMap();
- bucketNum = hash(key);
- child = pArray.get(bucketNum / CHILD_BUCKETS);
- if (growCount > 2) {
- //changeHashFunc();
- }
- growCount++;
- }
- occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, false);
- }
-
- public void upsert(int key, int value) {
- int growCount = 0;
- int bucketNum = hash(key);
- ChildIntArrayManager child = pArray.get(bucketNum / CHILD_BUCKETS);
- while (child.isFull(bucketNum % CHILD_BUCKETS)) {
- growHashMap();
- bucketNum = hash(key);
- child = pArray.get(bucketNum / CHILD_BUCKETS);
- if (growCount > 2) {
- //changeHashFunc();
- }
- growCount++;
- }
- occupiedSlots += child.put(bucketNum % CHILD_BUCKETS, key, value, true);
- }
-
- private int hash(int key) {
- return key % hashMod;
- }
-
- private void growHashMap() {
- int size = pArray.size();
- int i;
-
- //grow buckets by adding more child
- for (i = 0; i < size; i++) {
- pArray.add(new ChildIntArrayManager(this));
- }
-
- //increase hashMod
- hashMod *= 2;
-
- //re-hash
- rehash(0, size, hashMod / 2);
- }
-
- private void shrinkHashMap() {
- int size = pArray.size();
- int i;
-
- //decrease hashMod
- hashMod /= 2;
-
- //re-hash
- rehash(size / 2, size, hashMod * 2);
-
- //shrink buckets by removing child(s)
- for (i = size - 1; i >= size / 2; i--) {
- pArray.remove(i);
- }
- }
-
- private void rehash(int begin, int end, int oldHashMod) {
- int i, j, k;
- int key, value;
- ChildIntArrayManager child;
-
- //re-hash
- for (i = begin; i < end; i++) {
- child = pArray.get(i);
- for (j = 0; j < CHILD_BUCKETS; j++) {
- if (child.cArray[j][0] == 0) {
- continue;
- }
- for (k = 1; k < NUM_OF_SLOTS; k++) {
- //if the hashValue of the key is different, then re-hash it.
- key = child.cArray[j][k * 2];
- if (hash(key) != key % oldHashMod) {
- value = child.cArray[j][k * 2 + 1];
- //remove existing key and value
- //Notice! To avoid bucket iteration, child.remove() is not used.
- child.cArray[j][k * 2] = -1;
- child.cArray[j][0]--;
- //re-hash it
- pArray.get(hash(key) / CHILD_BUCKETS).put(hash(key) % CHILD_BUCKETS, key, value, false);
- }
- }
- }
- }
- }
-
- // private void changeHashFunc() {
- // //TODO need to implement.
- // throw new UnsupportedOperationException("changeHashFunc() not implemented");
- // }
-
- public int get(int key) {
- int bucketNum = hash(key);
- return pArray.get(bucketNum / CHILD_BUCKETS).get(bucketNum % CHILD_BUCKETS, key);
- }
-
- public void remove(int key) {
- int bucketNum = hash(key);
- occupiedSlots -= pArray.get(bucketNum / CHILD_BUCKETS).remove(bucketNum % CHILD_BUCKETS, key);
-
- if (needShrink()) {
- shrinkHashMap();
- }
- }
-
- /**
- * Shrink policy:
- * Shrink when the resource under-utilization lasts for a certain amount of time.
- *
- * @return
- */
- private boolean needShrink() {
- int size = pArray.size();
- int usedSlots = occupiedSlots;
- if (usedSlots == 0) {
- usedSlots = 1;
- }
- if (size > 1 && size * CHILD_BUCKETS * NUM_OF_SLOTS / usedSlots >= 3 && isSafeToShrink()) {
- if (isShrinkTimerOn) {
- if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
- isShrinkTimerOn = false;
- return true;
- }
- } else {
- //turn on timer
- isShrinkTimerOn = true;
- shrinkTimer = System.currentTimeMillis();
- }
- } else {
- //turn off timer
- isShrinkTimerOn = false;
- }
- return false;
- }
-
- private boolean isSafeToShrink() {
- int i, j;
- int size = pArray.size();
- //Child: 0, 1, 2, 3, 4, 5, 6, 7
- //[HChild(Head Child):0 and TChild(Tail Child): 4], [1(H),5(T)], [2(H),6(T)] and so on.
- //When the map shrinks, the sum of occupied slots in H/TChild should not exceed the NUM_OF_SLOTS-1.
- //Then it is safe to shrink. Otherwise, unsafe.
- ChildIntArrayManager HChild, TChild;
-
- for (i = 0; i < size / 2; i++) {
- HChild = pArray.get(i);
- TChild = pArray.get(size / 2 + i);
- for (j = 0; j < CHILD_BUCKETS; j++) {
- if (HChild.cArray[j][0] + TChild.cArray[j][0] > NUM_OF_SLOTS - 1) {
- return false;
- }
- }
- }
- return true;
- }
-
- public String prettyPrint() {
- StringBuilder s = new StringBuilder("\n########### PrimitiveIntHashMap Status #############\n");
- ChildIntArrayManager child;
- int i, j, k;
- int size = pArray.size();
- for (i = 0; i < size; i++) {
- child = pArray.get(i);
- s.append("child[").append(i).append("]\n");
- for (j = 0; j < CHILD_BUCKETS; j++) {
- s.append(j).append(" ");
- for (k = 0; k < NUM_OF_SLOTS; k++) {
- s.append("[").append(child.cArray[j][k * 2]).append(",").append(child.cArray[j][k * 2 + 1])
- .append("] ");
- }
- s.append("\n");
- }
- }
- return s.toString();
- }
-
- public int getNumOfSlots() {
- return NUM_OF_SLOTS;
- }
-
- public int getNumOfChildBuckets() {
- return CHILD_BUCKETS;
- }
-
- public void clear(boolean needShrink) {
- int size = pArray.size();
- for (int i = size - 1; i >= 0; i--) {
- if (needShrink && i != 0) {
- pArray.remove(i);
- } else {
- pArray.get(i).clear();
- }
- }
- occupiedSlots = 0;
- }
-
- ///////////////////////////////////////
- // iterate method
- ///////////////////////////////////////
-
- public void beginIterate() {
- iterChildIndex = 0;
- iterBucketIndex = 0;
- iterSlotIndex = 1;
- }
-
- public KeyValuePair getNextKeyValue() {
- for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
- for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
- if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
- continue;
- }
- for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
- iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
- if (iterPair.key == -1) {
- continue;
- }
- iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
- iterSlotIndex++;
- return iterPair;
- }
- }
- }
- return null;
- }
-
- public int getNextKey() {
- for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
- for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
- if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
- continue;
- }
- for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
- iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
- if (iterPair.key == -1) {
- continue;
- }
- iterSlotIndex++;
- return iterPair.key;
- }
- }
- }
- return -1;
- }
-
- public int getNextValue() {
- for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
- for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
- if (iterSlotIndex == 1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
- continue;
- }
- for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
- iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2];
- if (iterPair.key == -1) {
- continue;
- }
- iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex * 2 + 1];
- iterSlotIndex++;
- return iterPair.value;
- }
- }
- }
- return -1;
- }
-
- public static class KeyValuePair {
- public int key;
- public int value;
- }
-}
-
-class ChildIntArrayManager {
- private final int DIM1_SIZE;
- private final int DIM2_SIZE;
- private final int NUM_OF_SLOTS;
- public int[][] cArray; //child array
-
- public ChildIntArrayManager(PrimitiveIntHashMap parentHashMap) {
- DIM1_SIZE = parentHashMap.getNumOfChildBuckets();
- DIM2_SIZE = parentHashMap.getNumOfSlots() * 2; //2: Array of [key, value] pair
- NUM_OF_SLOTS = parentHashMap.getNumOfSlots();
- initialize();
- }
-
- private void initialize() {
- cArray = new int[DIM1_SIZE][DIM2_SIZE];
- int i, j;
- for (i = 0; i < DIM1_SIZE; i++) {
- //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
- //cArray[i][1] is not used.
- cArray[i][0] = 0;
- for (j = 1; j < NUM_OF_SLOTS; j++) {
- cArray[i][j * 2] = -1; // -1 represent that the slot is empty
- }
- }
- }
-
- public void clear() {
- int i, j;
- for (i = 0; i < DIM1_SIZE; i++) {
- //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
- //cArray[i][1] is not used.
- if (cArray[i][0] == 0) {
- continue;
- }
- cArray[i][0] = 0;
- for (j = 1; j < NUM_OF_SLOTS; j++) {
- cArray[i][j * 2] = -1; // -1 represent that the slot is empty
- }
- }
- }
-
- public void deinitialize() {
- cArray = null;
- }
-
- public void allocate() {
- initialize();
- }
-
- public boolean isFull(int bucketNum) {
- return cArray[bucketNum][0] == NUM_OF_SLOTS - 1;
- }
-
- public boolean isEmpty(int bucketNum) {
- return cArray[bucketNum][0] == 0;
- }
-
- /**
- * Put key,value into a slot in the bucket if the key doesn't exist.
- * Update value if the key exists and if isUpsert is true
- * No need to call get() to check the existence of the key before calling put().
- * Notice! Caller should make sure that there is an available slot.
- *
- * @param bucketNum
- * @param key
- * @param value
- * @param isUpsert
- * @return 1 for new insertion, 0 for key duplication
- */
- public int put(int bucketNum, int key, int value, boolean isUpsert) {
- int i;
- int emptySlot = -1;
-
- if (cArray[bucketNum][0] == 0) {
- cArray[bucketNum][2] = key;
- cArray[bucketNum][3] = value;
- cArray[bucketNum][0]++;
- return 1;
- }
-
- for (i = 1; i < NUM_OF_SLOTS; i++) {
- if (cArray[bucketNum][i * 2] == key) {
- if (isUpsert) {
- cArray[bucketNum][i * 2 + 1] = value;
- }
- return 0;
- } else if (cArray[bucketNum][i * 2] == -1) {
- emptySlot = i;
- }
- }
-
- if (emptySlot == -1) {
- throw new UnsupportedOperationException("error");
- }
-
- cArray[bucketNum][emptySlot * 2] = key;
- cArray[bucketNum][emptySlot * 2 + 1] = value;
- cArray[bucketNum][0]++;
- return 1;
- }
-
- public int get(int bucketNum, int key) {
- int i;
-
- if (cArray[bucketNum][0] == 0) {
- return -1;
- }
-
- for (i = 1; i < NUM_OF_SLOTS; i++) {
- if (cArray[bucketNum][i * 2] == key) {
- return cArray[bucketNum][i * 2 + 1];
- }
- }
- return -1;
- }
-
- /**
- * remove key if it exists. Otherwise, ignore it.
- *
- * @param bucketNum
- * @param key
- * @return 1 for success, 0 if the key doesn't exist
- */
- public int remove(int bucketNum, int key) {
- int i;
-
- if (cArray[bucketNum][0] == 0) {
- return 0;
- }
-
- for (i = 1; i < NUM_OF_SLOTS; i++) {
- if (cArray[bucketNum][i * 2] == key) {
- cArray[bucketNum][i * 2] = -1;
- cArray[bucketNum][0]--;
- return 1;
- }
- }
-
- return 0;
- }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 21268e5..4085fb4 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -89,8 +89,7 @@
public void append(ILogRecord logRecord, long appendLsn) {
logRecord.writeLogRecord(appendBuffer);
- if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
- && logRecord.getLogType() != LogType.WAIT) {
+ if (isLocalTransactionLog(logRecord)) {
logRecord.getTxnCtx().setLastLSN(appendLsn);
}
@@ -100,13 +99,10 @@
LOGGER.info("append()| appendOffset: " + appendOffset);
}
if (logRecord.getLogSource() == LogSource.LOCAL) {
- if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
- || logRecord.getLogType() == LogType.WAIT) {
+ if (syncPendingNonFlushLog(logRecord)) {
logRecord.isFlushed(false);
syncCommitQ.add(logRecord);
- }
- if (logRecord.getLogType() == LogType.FLUSH) {
- logRecord.isFlushed(false);
+ } else if (logRecord.getLogType() == LogType.FLUSH) {
flushQ.add(logRecord);
}
} else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
@@ -117,6 +113,16 @@
}
}
+ private boolean syncPendingNonFlushLog(ILogRecord logRecord) {
+ return logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+ || logRecord.getLogType() == LogType.WAIT || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES;
+ }
+
+ private boolean isLocalTransactionLog(ILogRecord logRecord) {
+ return logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
+ && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
+ }
+
@Override
public void setFileChannel(FileChannel fileChannel) {
this.fileChannel = fileChannel;
@@ -231,7 +237,8 @@
notifyJobTermination();
} else if (logRecord.getLogType() == LogType.FLUSH) {
notifyFlushTermination();
- } else if (logRecord.getLogType() == LogType.WAIT) {
+ } else if (logRecord.getLogType() == LogType.WAIT
+ || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
notifyWaitTermination();
}
} else if (logRecord.getLogSource() == LogSource.REMOTE && (logRecord.getLogType() == LogType.JOB_COMMIT
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index 736de07..be227ec 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -32,7 +32,6 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
-import java.util.HashMap;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
@@ -134,13 +133,33 @@
@Override
public void log(ILogRecord logRecord) {
- if (logRecord.getLogType() == LogType.FLUSH) {
- flushLogsQ.add(logRecord);
- return;
+ if (!logToFlushQueue(logRecord)) {
+ appendToLogTail(logRecord);
}
- appendToLogTail(logRecord);
}
+ @SuppressWarnings("squid:S2445")
+ protected boolean logToFlushQueue(ILogRecord logRecord) {
+ //Remote flush logs do not need to be flushed separately since they may not trigger local flush
+ if ((logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL)
+ || logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+ logRecord.isFlushed(false);
+ flushLogsQ.add(logRecord);
+ if (logRecord.getLogType() == LogType.WAIT_FOR_FLUSHES) {
+ InvokeUtil.doUninterruptibly(() -> {
+ synchronized (logRecord) {
+ while (!logRecord.isFlushed()) {
+ logRecord.wait();
+ }
+ }
+ });
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @SuppressWarnings("squid:S2445")
protected void appendToLogTail(ILogRecord logRecord) {
syncAppendToLogTail(logRecord);
if (waitForFlush(logRecord) && !logRecord.isFlushed()) {
@@ -161,7 +180,7 @@
synchronized void syncAppendToLogTail(ILogRecord logRecord) {
if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
- && logRecord.getLogType() != LogType.WAIT) {
+ && logRecord.getLogType() != LogType.WAIT && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES) {
ITransactionContext txnCtx = logRecord.getTxnCtx();
if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
throw new ACIDException(
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index 1e13883..d2e9629 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -40,9 +40,11 @@
super(txnSubsystem);
}
+ @SuppressWarnings("squid:S2445")
@Override
public void log(ILogRecord logRecord) {
- boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT;
+ boolean shouldReplicate = logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT
+ && logRecord.getLogType() != LogType.WAIT_FOR_FLUSHES;
if (shouldReplicate) {
switch (logRecord.getLogType()) {
case LogType.ENTITY_COMMIT:
@@ -63,16 +65,12 @@
}
}
logRecord.setReplicate(shouldReplicate);
-
- //Remote flush logs do not need to be flushed separately since they may not trigger local flush
- if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
- flushLogsQ.add(logRecord);
- return;
+ if (!logToFlushQueue(logRecord)) {
+ appendToLogTail(logRecord);
}
-
- appendToLogTail(logRecord);
}
+ @SuppressWarnings("squid:S2445")
@Override
protected void appendToLogTail(ILogRecord logRecord) {
syncAppendToLogTail(logRecord);
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index 6e87cf0..f5ab359 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -802,6 +802,7 @@
<module>asterix-active</module>
<module>asterix-client-helper</module>
<module>asterix-license</module>
+ <module>asterix-geo</module>
</modules>
<dependencyManagement>
@@ -1206,6 +1207,11 @@
<artifactId>commons-codec</artifactId>
<version>1.9</version>
</dependency>
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ <version>8.1.1</version>
+ </dependency>
</dependencies>
</dependencyManagement>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 6fdbb91..fa00886 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -207,4 +207,26 @@
</properties>
</project>
</supplement>
+ <supplement>
+ <project>
+ <groupId>com.esri.geometry</groupId>
+ <artifactId>esri-geometry-api</artifactId>
+ <properties>
+ <!-- esri is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>2.0.0</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.0.0</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ <properties>
+ <!-- fastutil is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>8.1.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>8.1.1</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
</supplementalDataModels>
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index adb6c79..54292c3 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -21,9 +21,10 @@
import java.io.Serializable;
import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IFormattedException;
import org.apache.hyracks.api.util.ErrorMessageUtil;
-public class AlgebricksException extends Exception {
+public class AlgebricksException extends Exception implements IFormattedException {
private static final long serialVersionUID = 1L;
public static final int UNKNOWN = 0;
@@ -93,10 +94,12 @@
return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), params);
}
+ @Override
public String getComponent() {
return component;
}
+ @Override
public int getErrorCode() {
return errorCode;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 339cb15..18aa104 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -36,10 +36,12 @@
import org.junit.Test;
import org.junit.runner.RunWith;
import org.powermock.api.mockito.PowerMockito;
+import org.powermock.core.classloader.annotations.PowerMockIgnore;
import org.powermock.core.classloader.annotations.PrepareForTest;
import org.powermock.modules.junit4.PowerMockRunner;
@RunWith(PowerMockRunner.class)
+@PowerMockIgnore("javax.management.*")
@PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
public class SequentialFirstRuleCheckFixpointRuleControllerTest {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/pom.xml b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
index 669988c..2a97b6f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-core/pom.xml
@@ -78,6 +78,11 @@
<version>${project.version}</version>
</dependency>
<dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-storage-am-common</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
deleted file mode 100644
index 3ecdb35..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/ILogicalPlanAndMetadata.java
+++ /dev/null
@@ -1,30 +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.hyracks.algebricks.core.algebra.base;
-
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
-
-public interface ILogicalPlanAndMetadata {
- public ILogicalPlan getPlan();
-
- public IMetadataProvider<?, ?> getMetadataProvider();
-
- public AlgebricksPartitionConstraint getClusterLocations();
-}
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 6860147..ae66ee2 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
@@ -36,6 +36,7 @@
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
public interface IMetadataProvider<S, I> {
public IDataSource<S> findDataSource(S id) throws AlgebricksException;
@@ -47,7 +48,8 @@
*/
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
+ List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars,
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, IOperatorSchema opSchema,
IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
throws AlgebricksException;
@@ -206,5 +208,9 @@
List<LogicalVariable> prevSecondaryKeys, LogicalVariable prevAdditionalFilteringKeys,
RecordDescriptor inputDesc, JobGenContext context, JobSpecification spec) throws AlgebricksException;
+ public ITupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
+ IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
+ throws AlgebricksException;
+
public Map<String, String> getConfig();
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
index 2511fa9..6f11dc1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DataSourceScanOperator.java
@@ -35,7 +35,7 @@
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
public class DataSourceScanOperator extends AbstractDataSourceOperator {
- private List<LogicalVariable> projectVars;
+ private final List<LogicalVariable> projectVars;
private boolean projectPushed = false;
@@ -43,9 +43,22 @@
private List<LogicalVariable> minFilterVars;
private List<LogicalVariable> maxFilterVars;
+ // the select condition in the SELECT operator. Only results satisfying this selectCondition
+ // would be returned by this operator
+ private Mutable<ILogicalExpression> selectCondition;
+ // the maximum of number of results output by this operator
+ private long outputLimit = -1;
+
public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource) {
+ this(variables, dataSource, null, -1);
+ }
+
+ public DataSourceScanOperator(List<LogicalVariable> variables, IDataSource<?> dataSource,
+ Mutable<ILogicalExpression> selectCondition, long outputLimit) {
super(variables, dataSource);
projectVars = new ArrayList<LogicalVariable>();
+ this.selectCondition = selectCondition;
+ this.outputLimit = outputLimit;
}
@Override
@@ -133,4 +146,20 @@
public List<Mutable<ILogicalExpression>> getAdditionalFilteringExpressions() {
return additionalFilteringExpressions;
}
+
+ public Mutable<ILogicalExpression> getSelectCondition() {
+ return selectCondition;
+ }
+
+ public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+ this.selectCondition = selectCondition;
+ }
+
+ public long getOutputLimit() {
+ return outputLimit;
+ }
+
+ public void setOutputLimit(long outputLimit) {
+ this.outputLimit = outputLimit;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index e7fb6c0..c4bcc52 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -32,9 +32,23 @@
public class UnnestMapOperator extends AbstractUnnestMapOperator {
+ // the select condition in the SELECT operator. Only results satisfying this selectCondition
+ // would be returned by this operator
+ private Mutable<ILogicalExpression> selectCondition;
+ // the maximum of number of results output by this operator
+ private long outputLimit = -1;
+
public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
List<Object> variableTypes, boolean propagateInput) {
+ this(variables, expression, variableTypes, propagateInput, null, -1);
+ }
+
+ public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
+ List<Object> variableTypes, boolean propagateInput, Mutable<ILogicalExpression> selectCondition,
+ long outputLimit) {
super(variables, expression, variableTypes, propagateInput);
+ this.selectCondition = selectCondition;
+ this.outputLimit = outputLimit;
}
@Override
@@ -64,4 +78,20 @@
return env;
}
+ public Mutable<ILogicalExpression> getSelectCondition() {
+ return selectCondition;
+ }
+
+ public void setSelectCondition(Mutable<ILogicalExpression> selectCondition) {
+ this.selectCondition = selectCondition;
+ }
+
+ public long getOutputLimit() {
+ return outputLimit;
+ }
+
+ public void setOutputLimit(long outputLimit) {
+ this.outputLimit = outputLimit;
+ }
+
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index 74afdf5..d6062ee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -317,8 +317,10 @@
@Override
public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, ILogicalOperator arg)
throws AlgebricksException {
- DataSourceScanOperator opCopy =
- new DataSourceScanOperator(deepCopyVariableList(op.getVariables()), op.getDataSource());
+ Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+ ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
+ DataSourceScanOperator opCopy = new DataSourceScanOperator(deepCopyVariableList(op.getVariables()),
+ op.getDataSource(), newSelectCondition, op.getOutputLimit());
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
@@ -535,9 +537,11 @@
@Override
public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg)
throws AlgebricksException {
+ Mutable<ILogicalExpression> newSelectCondition = op.getSelectCondition() != null
+ ? exprDeepCopyVisitor.deepCopyExpressionReference(op.getSelectCondition()) : null;
UnnestMapOperator opCopy = new UnnestMapOperator(deepCopyVariableList(op.getVariables()),
exprDeepCopyVisitor.deepCopyExpressionReference(op.getExpressionRef()), op.getVariableTypes(),
- op.propagatesInput());
+ op.propagatesInput(), newSelectCondition, op.getOutputLimit());
deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
return opCopy;
}
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 600714b..0db0f74 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
@@ -35,11 +35,11 @@
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.DataSourceScanOperator;
+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.DistributeResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -225,8 +225,10 @@
public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
ArrayList<LogicalVariable> newInputList = new ArrayList<>();
newInputList.addAll(op.getVariables());
+ Mutable<ILogicalExpression> newSelectCondition =
+ op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
- new ArrayList<>(op.getVariableTypes()), op.propagatesInput());
+ new ArrayList<>(op.getVariableTypes()), op.propagatesInput(), newSelectCondition, op.getOutputLimit());
}
@Override
@@ -242,7 +244,11 @@
public ILogicalOperator visitDataScanOperator(DataSourceScanOperator op, Void arg) throws AlgebricksException {
ArrayList<LogicalVariable> newInputList = new ArrayList<>();
newInputList.addAll(op.getVariables());
- return new DataSourceScanOperator(newInputList, op.getDataSource());
+ Mutable<ILogicalExpression> newSelectCondition =
+ op.getSelectCondition() != null ? deepCopyExpressionRef(op.getSelectCondition()) : null;
+ DataSourceScanOperator newOp =
+ new DataSourceScanOperator(newInputList, op.getDataSource(), newSelectCondition, op.getOutputLimit());
+ return newOp;
}
@Override
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 cf24ee7..3587e29 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
@@ -34,11 +34,11 @@
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.DataSourceScanOperator;
+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.DistributeResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -134,6 +134,9 @@
}
}
substVarTypes(op, pair);
+ if (op.getSelectCondition() != null) {
+ op.getSelectCondition().getValue().substituteVar(pair.first, pair.second);
+ }
return null;
}
@@ -316,6 +319,9 @@
public Void visitUnnestMapOperator(UnnestMapOperator op, Pair<LogicalVariable, LogicalVariable> pair)
throws AlgebricksException {
substituteVarsForAbstractUnnestMapOp(op, pair);
+ if (op.getSelectCondition() != null) {
+ op.getSelectCondition().getValue().substituteVar(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 b8cb4ff..d57a998 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
@@ -19,7 +19,9 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -34,11 +36,11 @@
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.DataSourceScanOperator;
+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.DistributeResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -76,7 +78,7 @@
public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
- private Collection<LogicalVariable> usedVariables;
+ private final Collection<LogicalVariable> usedVariables;
public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
this.usedVariables = usedVariables;
@@ -105,6 +107,12 @@
e.getValue().getUsedVariables(usedVariables);
}
}
+ if (op.getSelectCondition() != null) {
+ Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+ op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+ usedVariablesBySelect.removeAll(op.getVariables());
+ usedVariables.addAll(usedVariablesBySelect);
+ }
return null;
}
@@ -305,6 +313,12 @@
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Void arg) {
getUsedVarsFromExprAndFilterExpr(op);
+ if (op.getSelectCondition() != null) {
+ Set<LogicalVariable> usedVariablesBySelect = new HashSet<>();
+ op.getSelectCondition().getValue().getUsedVariables(usedVariablesBySelect);
+ usedVariablesBySelect.removeAll(op.getVariables());
+ usedVariables.addAll(usedVariablesBySelect);
+ }
return null;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 1421cef..e8c5c64 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -43,11 +43,12 @@
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
@SuppressWarnings("rawtypes")
public class DataSourceScanPOperator extends AbstractScanPOperator {
- private IDataSource<?> dataSource;
+ private final IDataSource<?> dataSource;
private Object implConfig;
public DataSourceScanPOperator(IDataSource<?> dataSource) {
@@ -109,9 +110,15 @@
List<LogicalVariable> vars = scan.getVariables();
List<LogicalVariable> projectVars = scan.getProjectVariables();
- Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p =
- mp.getScannerRuntime(dataSource, vars, projectVars, scan.isProjectPushed(), scan.getMinFilterVars(),
- scan.getMaxFilterVars(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
+ ITupleFilterFactory tupleFilterFactory = null;
+ if (scan.getSelectCondition() != null) {
+ tupleFilterFactory = context.getMetadataProvider().createTupleFilterFactory(
+ new IOperatorSchema[] { opSchema }, typeEnv, scan.getSelectCondition().getValue(), context);
+ }
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars, projectVars,
+ scan.isProjectPushed(), scan.getMinFilterVars(), scan.getMaxFilterVars(), tupleFilterFactory,
+ scan.getOutputLimit(), opSchema, typeEnv, context, builder.getJobSpec(), implConfig);
builder.contributeHyracksOperator(scan, p.first);
if (p.second != null) {
builder.contributeAlgebricksPartitionConstraint(p.first, p.second);
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 70f19c1..99ed738 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
@@ -299,21 +299,25 @@
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
- return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+ appendLimitInformation(plan, op.getOutputLimit());
+ return null;
}
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
throws AlgebricksException {
- return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ return null;
}
- private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
- throws AlgebricksException {
+ private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+ String opSignature) throws AlgebricksException {
AlgebricksAppendable plan = addIndent(indent).append(opSignature + " " + op.getVariables() + " <- "
+ op.getExpressionRef().getValue().accept(exprVisitor, indent));
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
- return null;
+ return plan;
}
@Override
@@ -321,6 +325,24 @@
AlgebricksAppendable plan = addIndent(indent).append(
"data-scan " + op.getProjectVariables() + "<-" + op.getVariables() + " <- " + op.getDataSource());
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars());
+ appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+ appendLimitInformation(plan, op.getOutputLimit());
+ return null;
+ }
+
+ private Void appendSelectConditionInformation(AlgebricksAppendable plan,
+ Mutable<ILogicalExpression> selectCondition, Integer indent) throws AlgebricksException {
+ if (selectCondition != null) {
+ plan.append(" condition (").append(selectCondition.getValue().accept(exprVisitor, indent)).append(")");
+ }
+
+ return null;
+ }
+
+ private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit) throws AlgebricksException {
+ if (outputLimit >= 0) {
+ plan.append(" limit ").append(String.valueOf(outputLimit));
+ }
return null;
}
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 8acf08b..f1f1f3b 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
@@ -83,7 +83,7 @@
public class IdCounter {
private int id;
- private Deque<Integer> prefix;
+ private final Deque<Integer> prefix;
public IdCounter() {
prefix = new LinkedList<Integer>();
@@ -400,24 +400,28 @@
@Override
public Void visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
- return printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ AlgebricksAppendable plan = printAbstractUnnestMapOperator(op, indent, "unnest-map");
+ appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+ appendLimitInformation(plan, op.getOutputLimit(), indent);
+ return null;
}
@Override
public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Integer indent)
throws AlgebricksException {
- return printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ printAbstractUnnestMapOperator(op, indent, "left-outer-unnest-map");
+ return null;
}
- private Void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent, String opSignature)
- throws AlgebricksException {
+ private AlgebricksAppendable printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, Integer indent,
+ String opSignature) throws AlgebricksException {
AlgebricksAppendable plan = addIndent(indent).append("\"operator\": \"" + opSignature + "\"");
variablePrintHelper(op.getVariables(), indent);
buffer.append(",\n");
addIndent(indent).append("\"expressions\": \""
+ op.getExpressionRef().getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
- return null;
+ return plan;
}
@Override
@@ -435,6 +439,8 @@
addIndent(indent).append("\"data-source\": \"" + op.getDataSource() + "\"");
}
appendFilterInformation(plan, op.getMinFilterVars(), op.getMaxFilterVars(), indent);
+ appendSelectConditionInformation(plan, op.getSelectCondition(), indent);
+ appendLimitInformation(plan, op.getOutputLimit(), indent);
return null;
}
@@ -467,6 +473,25 @@
return null;
}
+ private Void appendSelectConditionInformation(AlgebricksAppendable plan, Mutable<ILogicalExpression> condition,
+ Integer indent) throws AlgebricksException {
+ if (condition != null) {
+ plan.append(",\n");
+ addIndent(indent).append(
+ "\"condition\": \"" + condition.getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+ }
+ return null;
+ }
+
+ private Void appendLimitInformation(AlgebricksAppendable plan, long outputLimit, Integer indent)
+ throws AlgebricksException {
+ if (outputLimit >= 0) {
+ plan.append(",\n");
+ addIndent(indent).append("\"limit\": \"" + outputLimit + "\"");
+ }
+ return null;
+ }
+
private void appendVars(List<LogicalVariable> minFilterVars) throws AlgebricksException {
boolean first = true;
for (LogicalVariable v : minFilterVars) {
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 4649d6d..2cb2d35 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
@@ -289,21 +289,24 @@
@Override
public String visitUnnestMapOperator(UnnestMapOperator op, Void noArgs) throws AlgebricksException {
stringBuilder.setLength(0);
- return printAbstractUnnestMapOperator(op, "unnest-map");
+ printAbstractUnnestMapOperator(op, "unnest-map");
+ appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+ appendLimitInformation(stringBuilder, op.getOutputLimit());
+ return stringBuilder.toString();
}
@Override
public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void noArgs)
throws AlgebricksException {
stringBuilder.setLength(0);
- return printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+ printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+ return stringBuilder.toString();
}
- private String printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
+ private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
stringBuilder.append(opSignature).append(" ").append(op.getVariables()).append(" <- ")
.append(op.getExpressionRef().getValue().toString());
appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
- return stringBuilder.toString();
}
@Override
@@ -312,10 +315,12 @@
stringBuilder.append("data-scan ").append(op.getProjectVariables()).append("<-").append(op.getVariables())
.append(" <- ").append(op.getDataSource());
appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
+ appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
+ appendLimitInformation(stringBuilder, op.getOutputLimit());
return stringBuilder.toString();
}
- private String appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
+ private void appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
List<LogicalVariable> maxFilterVars) {
if (minFilterVars != null || maxFilterVars != null) {
plan.append(" with filter on");
@@ -326,7 +331,21 @@
if (maxFilterVars != null) {
plan.append(" max:").append(maxFilterVars);
}
- return stringBuilder.toString();
+ }
+
+ private Void appendSelectConditionInformation(StringBuilder plan, Mutable<ILogicalExpression> condition)
+ throws AlgebricksException {
+ if (condition != null) {
+ plan.append(" condition:").append(condition.getValue().toString());
+ }
+ return null;
+ }
+
+ private Void appendLimitInformation(StringBuilder plan, long outputLimit) throws AlgebricksException {
+ if (outputLimit >= 0) {
+ plan.append(" limit:").append(String.valueOf(outputLimit));
+ }
+ return null;
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
index 24a5e21..d507e5a 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryBooleanInspector.java
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.algebricks.data;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
public interface IBinaryBooleanInspector {
- public boolean getBooleanValue(byte[] bytes, int offset, int length);
+ public boolean getBooleanValue(byte[] bytes, int offset, int length) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
index 23c41fe..ad2e77a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -39,8 +39,9 @@
GET_JOB_STATUS,
GET_JOB_INFO,
START_JOB,
- DISTRIBUTE_JOB,
- DESTROY_JOB,
+ DEPLOY_JOB,
+ UNDEPLOY_JOB,
+ REDEPLOY_JOB,
CANCEL_JOB,
GET_DATASET_DIRECTORY_SERIVICE_INFO,
GET_DATASET_RESULT_STATUS,
@@ -107,6 +108,32 @@
}
}
+ public static class redeployJobSpecFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final byte[] acggfBytes;
+
+ private final DeployedJobSpecId deployedJobSpecId;
+
+ public redeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) {
+ this.deployedJobSpecId = deployedJobSpecId;
+ this.acggfBytes = acggfBytes;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REDEPLOY_JOB;
+ }
+
+ public byte[] getACGGFBytes() {
+ return acggfBytes;
+ }
+
+ public DeployedJobSpecId getDeployedJobSpecId() {
+ return deployedJobSpecId;
+ }
+ }
+
public static class DeployJobSpecFunction extends Function {
private static final long serialVersionUID = 1L;
@@ -118,7 +145,7 @@
@Override
public FunctionId getFunctionId() {
- return FunctionId.DISTRIBUTE_JOB;
+ return FunctionId.DEPLOY_JOB;
}
public byte[] getACGGFBytes() {
@@ -159,7 +186,7 @@
@Override
public FunctionId getFunctionId() {
- return FunctionId.DESTROY_JOB;
+ return FunctionId.UNDEPLOY_JOB;
}
public DeployedJobSpecId getDeployedJobSpecId() {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index eddcaa5..1ee9bd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -98,10 +98,17 @@
}
@Override
- public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+ public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception {
+ HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+ new HyracksClientInterfaceFunctions.redeployJobSpecFunction(deployedJobSpecId, acggfBytes);
+ rpci.call(ipcHandle, udjsf);
+ }
+
+ @Override
+ public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
HyracksClientInterfaceFunctions.UndeployJobSpecFunction sjf =
new HyracksClientInterfaceFunctions.UndeployJobSpecFunction(deployedJobSpecId);
- return (DeployedJobSpecId) rpci.call(ipcHandle, sjf);
+ rpci.call(ipcHandle, sjf);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
index f676d27..f635d94 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
@@ -111,6 +111,13 @@
}
@Override
+ public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception {
+ JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
+ new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
+ hci.redeployJobSpec(deployedJobSpecId, JavaSerializationUtils.serialize(jsacggf));
+ }
+
+ @Override
public DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception {
JobSpecificationActivityClusterGraphGeneratorFactory jsacggf =
new JobSpecificationActivityClusterGraphGeneratorFactory(jobSpec);
@@ -118,8 +125,8 @@
}
@Override
- public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
- return hci.undeployJobSpec(deployedJobSpecId);
+ public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception {
+ hci.undeployJobSpec(deployedJobSpecId);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 510a6b6..b3b7677 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -99,13 +99,24 @@
DeployedJobSpecId deployJobSpec(JobSpecification jobSpec) throws Exception;
/**
+ * Update the JobSpec for a deployed job.
+ *
+ * @param deployedJobSpecId
+ * The id of the deployed job spec
+ * @param jobSpec
+ * Job Specification
+ * @throws Exception
+ */
+ void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, JobSpecification jobSpec) throws Exception;
+
+ /**
* Remove the deployed Job Spec
*
* @param deployedJobSpecId
* The id of the deployed job spec
* @throws Exception
*/
- DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+ void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
/**
* Used to run a deployed Job Spec by id
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
index f0c7872..6a75806 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientInterface.java
@@ -45,7 +45,9 @@
public DeployedJobSpecId deployJobSpec(byte[] acggfBytes) throws Exception;
- public DeployedJobSpecId undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
+ public void redeployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] acggfBytes) throws Exception;
+
+ public void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
deleted file mode 100644
index c6d8f71..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/IOperatorNodePullable.java
+++ /dev/null
@@ -1,23 +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.hyracks.api.dataflow;
-
-public interface IOperatorNodePullable extends IOpenableDataReader<Object[]> {
- public void setDataReader(int index, IOpenableDataWriter<Object[]> writer);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
deleted file mode 100644
index 4ec4c12..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunction.java
+++ /dev/null
@@ -1,25 +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.hyracks.api.dataflow.value;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IHashFunction<T> {
- public int hash(T o) throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
deleted file mode 100644
index b22dbe7..0000000
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IHashFunctionFactory.java
+++ /dev/null
@@ -1,25 +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.hyracks.api.dataflow.value;
-
-import java.io.Serializable;
-
-public interface IHashFunctionFactory<T> extends Serializable {
- public IHashFunction<T> createHashFunction();
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 51afac1..b6d7cc7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -121,7 +121,7 @@
public static final int FOUND_MULTIPLE_TRANSACTIONS = 85;
public static final int UNRECOGNIZED_INDEX_COMPONENT_FILE = 86;
public static final int UNEQUAL_NUM_FILTERS_TREES = 87;
- public static final int CANNOT_MODIFY_INDEX_DISK_IS_FULL = 88;
+ public static final int INDEX_NOT_MODIFIABLE = 88;
public static final int GROUP_BY_MEMORY_BUDGET_EXCEEDS = 89;
public static final int ILLEGAL_MEMORY_BUDGET = 90;
public static final int TIMEOUT = 91;
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index 210779e..bf1f9dc 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -23,7 +23,7 @@
import org.apache.hyracks.api.util.ErrorMessageUtil;
-public class HyracksException extends IOException {
+public class HyracksException extends IOException implements IFormattedException {
private static final long serialVersionUID = 1L;
public static final int UNKNOWN = 0;
@@ -112,10 +112,12 @@
this(component, errorCode, message, cause, null, params);
}
+ @Override
public String getComponent() {
return component;
}
+ @Override
public int getErrorCode() {
return errorCode;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
new file mode 100644
index 0000000..994915f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
@@ -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.
+ */
+package org.apache.hyracks.api.exceptions;
+
+public interface IFormattedException {
+
+ /**
+ * Gets the component of this {@link IFormattedException}
+ *
+ * @return the exception component
+ */
+ String getComponent();
+
+ /**
+ * Gets the error code of this {@link IFormattedException}
+ *
+ * @return the error code
+ */
+ int getErrorCode();
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 9302f46..bba0de7 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -119,4 +119,15 @@
stackThrowable.setStackTrace(thread.getStackTrace());
return stackThrowable;
}
+
+ public static Throwable getRootCause(Throwable e) {
+ Throwable current = e;
+ Throwable cause = e.getCause();
+ while (cause != null && cause != current) {
+ current = cause;
+ cause = current.getCause();
+ }
+ return current;
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
index ccf0163..85019c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/SingleThreadEventProcessor.java
@@ -65,20 +65,20 @@
}
}
- public void stop() throws HyracksDataException, InterruptedException {
+ public void stop() throws HyracksDataException {
stopped = true;
executorThread.interrupt();
- executorThread.join(1000);
+ InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
int attempt = 0;
while (executorThread.isAlive()) {
attempt++;
- LOGGER.log(Level.WARN,
- "Failed to stop event processor after " + attempt + " attempts. Interrupted exception swallowed?");
+ LOGGER.log(Level.WARN, "Failed to stop event processor after {} attempts. Interrupted exception swallowed?",
+ attempt, ExceptionUtils.fromThreadStack(executorThread));
if (attempt == 10) {
throw HyracksDataException.create(ErrorCode.FAILED_TO_SHUTDOWN_EVENT_PROCESSOR, name);
}
executorThread.interrupt();
- executorThread.join(1000);
+ InvokeUtil.doUninterruptibly(() -> executorThread.join(1000));
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index 452d379..ef07038 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -104,7 +104,7 @@
85 = Found more than one transaction file in %1$s
86 = Found an unrecognized index file %1$s
87 = Unequal number of trees and filters found in %1$s
-88 = Cannot modify index (Disk is full)
+88 = Cannot modify index (%1$s)
89 = The byte size of a single group (%1$s bytes) exceeds the budget for a group by operator (%2$s bytes)
90 = Memory budget for the %1$s operator (%2$s bytes) is lower than the minimum (%3$s bytes)
91 = Operation timed out
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
index e46aa7f..a669402 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/ClientInterfaceIPCI.java
@@ -85,13 +85,19 @@
ccs.getWorkQueue().schedule(new GetJobInfoWork(ccs.getJobManager(), gjif.getJobId(),
new IPCResponder<JobInfo>(handle, mid)));
break;
- case DISTRIBUTE_JOB:
+ case DEPLOY_JOB:
HyracksClientInterfaceFunctions.DeployJobSpecFunction djf =
(HyracksClientInterfaceFunctions.DeployJobSpecFunction) fn;
ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, djf.getACGGFBytes(),
- deployedJobSpecIdFactory.create(), new IPCResponder<>(handle, mid)));
+ deployedJobSpecIdFactory.create(), false, new IPCResponder<>(handle, mid)));
break;
- case DESTROY_JOB:
+ case REDEPLOY_JOB:
+ HyracksClientInterfaceFunctions.redeployJobSpecFunction udjsf =
+ (HyracksClientInterfaceFunctions.redeployJobSpecFunction) fn;
+ ccs.getWorkQueue().schedule(new DeployJobSpecWork(ccs, udjsf.getACGGFBytes(),
+ udjsf.getDeployedJobSpecId(), true, new IPCResponder<>(handle, mid)));
+ break;
+ case UNDEPLOY_JOB:
HyracksClientInterfaceFunctions.UndeployJobSpecFunction dsjf =
(HyracksClientInterfaceFunctions.UndeployJobSpecFunction) fn;
ccs.getWorkQueue().schedule(
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
index 0e22c25..041e224 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/DeployedJobSpecStore.java
@@ -40,9 +40,6 @@
public void addDeployedJobSpecDescriptor(DeployedJobSpecId deployedJobSpecId,
ActivityClusterGraph activityClusterGraph, JobSpecification jobSpecification,
Set<Constraint> activityClusterGraphConstraints) throws HyracksException {
- if (deployedJobSpecDescriptorMap.get(deployedJobSpecId) != null) {
- throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
- }
DeployedJobSpecDescriptor descriptor =
new DeployedJobSpecDescriptor(activityClusterGraph, jobSpecification, activityClusterGraphConstraints);
deployedJobSpecDescriptorMap.put(deployedJobSpecId, descriptor);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
index b44a6bb8..2b03324 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/cluster/NodeManager.java
@@ -105,9 +105,7 @@
failNode(nodeId);
}
try {
- // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
- IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(ncState.getNodeController().getAddress());
- ncIPCHandle.send(-1, new AbortCCJobsFunction(ccConfig.getCcId()), null);
+ ncState.getNodeController().abortJobs(ccs.getCcId());
} catch (IPCException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
index c51f3c5..60c88c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/DeployJobSpecWork.java
@@ -39,20 +39,24 @@
private final byte[] acggfBytes;
private final DeployedJobSpecId deployedJobSpecId;
private final IResultCallback<DeployedJobSpecId> callback;
+ private final boolean upsert;
public DeployJobSpecWork(ClusterControllerService ccs, byte[] acggfBytes, DeployedJobSpecId deployedJobSpecId,
- IResultCallback<DeployedJobSpecId> callback) {
+ boolean upsert, IResultCallback<DeployedJobSpecId> callback) {
this.deployedJobSpecId = deployedJobSpecId;
this.ccs = ccs;
this.acggfBytes = acggfBytes;
this.callback = callback;
+ this.upsert = upsert;
}
@Override
protected void doRun() throws Exception {
try {
final CCServiceContext ccServiceCtx = ccs.getContext();
- ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+ if (!upsert) {
+ ccs.getDeployedJobSpecStore().checkForExistingDeployedJobSpecDescriptor(deployedJobSpecId);
+ }
IActivityClusterGraphGeneratorFactory acggf =
(IActivityClusterGraphGeneratorFactory) DeploymentUtils.deserialize(acggfBytes, null, ccServiceCtx);
IActivityClusterGraphGenerator acgg =
@@ -65,7 +69,7 @@
INodeManager nodeManager = ccs.getNodeManager();
for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
- node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes);
+ node.getNodeController().deployJobSpec(deployedJobSpecId, acgBytes, upsert);
}
callback.setValue(deployedJobSpecId);
} catch (Exception e) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
index de7d941..00693df 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/RegisterNodeWork.java
@@ -28,11 +28,8 @@
import org.apache.hyracks.control.cc.cluster.INodeManager;
import org.apache.hyracks.control.common.controllers.NodeParameters;
import org.apache.hyracks.control.common.controllers.NodeRegistration;
-import org.apache.hyracks.control.common.ipc.CCNCFunctions;
import org.apache.hyracks.control.common.ipc.NodeControllerRemoteProxy;
import org.apache.hyracks.control.common.work.SynchronizableWork;
-import org.apache.hyracks.ipc.api.IIPCHandle;
-import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -52,36 +49,33 @@
@Override
protected void doRun() throws Exception {
String id = reg.getNodeId();
- // TODO(mblow): it seems we should close IPC handles when we're done with them (like here)
- IIPCHandle ncIPCHandle = ccs.getClusterIPC().getHandle(reg.getNodeControllerAddress());
- CCNCFunctions.NodeRegistrationResult result;
- Map<IOption, Object> ncConfiguration = new HashMap<>();
+ LOGGER.warn("Registering node: {}", id);
+ NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
+ ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
+ INodeManager nodeManager = ccs.getNodeManager();
try {
- LOGGER.log(Level.WARN, "Registering INodeController: id = " + id);
- NodeControllerRemoteProxy nc = new NodeControllerRemoteProxy(ccs.getCcId(),
- ccs.getClusterIPC().getReconnectingHandle(reg.getNodeControllerAddress()));
NodeControllerState state = new NodeControllerState(nc, reg);
- INodeManager nodeManager = ccs.getNodeManager();
nodeManager.addNode(id, state);
IApplicationConfig cfg = state.getNCConfig().getConfigManager().getNodeEffectiveConfig(id);
+ final Map<IOption, Object> ncConfiguration = new HashMap<>();
for (IOption option : cfg.getOptions()) {
ncConfiguration.put(option, cfg.get(option));
}
- LOGGER.log(Level.INFO, "Registered INodeController: id = " + id);
+ LOGGER.warn("Registered node: {}", id);
NodeParameters params = new NodeParameters();
params.setClusterControllerInfo(ccs.getClusterControllerInfo());
params.setDistributedState(ccs.getContext().getDistributedState());
params.setHeartbeatPeriod(ccs.getCCConfig().getHeartbeatPeriodMillis());
params.setProfileDumpPeriod(ccs.getCCConfig().getProfileDumpPeriod());
params.setRegistrationId(registrationId);
- result = new CCNCFunctions.NodeRegistrationResult(params, null);
+ LOGGER.warn("sending registration response to node {}", id);
+ nc.sendRegistrationResult(params, null);
+ LOGGER.warn("notifying node {} joined", id);
+ ccs.getContext().notifyNodeJoin(id, ncConfiguration);
} catch (Exception e) {
- LOGGER.log(Level.WARN, "Node registration failed", e);
- result = new CCNCFunctions.NodeRegistrationResult(null, e);
+ LOGGER.error("Node {} registration failed", id, e);
+ nodeManager.removeNode(id);
+ nc.sendRegistrationResult(null, e);
}
- LOGGER.warn("sending registration response to node");
- ncIPCHandle.send(-1, result, null);
- LOGGER.warn("notifying node join");
- ccs.getContext().notifyNodeJoin(id, ncConfiguration);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
index 143c8c1..69b55ed 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/UndeployJobSpecWork.java
@@ -28,10 +28,10 @@
public class UndeployJobSpecWork extends SynchronizableWork {
private final ClusterControllerService ccs;
private final DeployedJobSpecId deployedJobSpecId;
- private final IResultCallback<DeployedJobSpecId> callback;
+ private final IResultCallback<Void> callback;
public UndeployJobSpecWork(ClusterControllerService ccs, DeployedJobSpecId deployedJobSpecId,
- IResultCallback<DeployedJobSpecId> callback) {
+ IResultCallback<Void> callback) {
this.deployedJobSpecId = deployedJobSpecId;
this.ccs = ccs;
this.callback = callback;
@@ -45,7 +45,7 @@
for (NodeControllerState node : nodeManager.getAllNodeControllerStates()) {
node.getNodeController().undeployJobSpec(deployedJobSpecId);
}
- callback.setValue(deployedJobSpecId);
+ callback.setValue(null);
} catch (Exception e) {
callback.setException(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
index 9959e34..cc2ed46 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -134,7 +134,6 @@
IPCSystem ipcSystem = Mockito.mock(IPCSystem.class);
IIPCHandle ipcHandle = Mockito.mock(IIPCHandle.class);
Mockito.when(ccs.getClusterIPC()).thenReturn(ipcSystem);
- Mockito.when(ipcSystem.getHandle(Mockito.any())).thenReturn(ipcHandle);
Mockito.when(ipcSystem.getHandle(Mockito.any(), Mockito.anyInt())).thenReturn(ipcHandle);
Mockito.when(ccs.getExecutor()).thenReturn(Executors.newCachedThreadPool());
return ccs;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
index 92764a7..fa835f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/base/INodeController.java
@@ -24,6 +24,7 @@
import java.util.Set;
import org.apache.hyracks.api.comm.NetworkAddress;
+import org.apache.hyracks.api.control.CcId;
import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
import org.apache.hyracks.api.dataflow.TaskAttemptId;
import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy;
@@ -33,7 +34,9 @@
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.job.JobStatus;
import org.apache.hyracks.api.partitions.PartitionId;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
+import org.apache.hyracks.ipc.exceptions.IPCException;
public interface INodeController {
void startTasks(DeploymentId deploymentId, JobId jobId, byte[] planBytes,
@@ -51,7 +54,8 @@
void undeployBinary(DeploymentId deploymentId) throws Exception;
- void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception;
+ void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean checkForDuplicate)
+ throws Exception;
void undeployJobSpec(DeployedJobSpecId deployedJobSpecId) throws Exception;
@@ -62,4 +66,22 @@
void sendApplicationMessageToNC(byte[] data, DeploymentId deploymentId, String nodeId) throws Exception;
void takeThreadDump(String requestId) throws Exception;
+
+ /**
+ * Sends a request to this {@link INodeController} to abort all jobs
+ * started by cluster controller with id {@code ccId}
+ *
+ * @param ccId
+ * @throws IPCException
+ */
+ void abortJobs(CcId ccId) throws IPCException;
+
+ /**
+ * Sends node registration result to this {@link INodeController}.
+ *
+ * @param parameters
+ * @param regFailure
+ * @throws IPCException
+ */
+ void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
index 8e02936..dea5198 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/CCNCFunctions.java
@@ -103,8 +103,8 @@
SHUTDOWN_REQUEST,
SHUTDOWN_RESPONSE,
- DISTRIBUTE_JOB,
- DESTROY_JOB,
+ DEPLOY_JOB,
+ UNDEPLOY_JOB,
DEPLOYED_JOB_FAILURE,
STATE_DUMP_REQUEST,
@@ -713,15 +713,18 @@
private final byte[] acgBytes;
- public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, CcId ccId) {
+ private final boolean upsert;
+
+ public DeployJobSpecFunction(DeployedJobSpecId deployedJobSpecId, byte[] acgBytes, boolean upsert, CcId ccId) {
super(ccId);
this.deployedJobSpecId = deployedJobSpecId;
this.acgBytes = acgBytes;
+ this.upsert = upsert;
}
@Override
public FunctionId getFunctionId() {
- return FunctionId.DISTRIBUTE_JOB;
+ return FunctionId.DEPLOY_JOB;
}
public DeployedJobSpecId getDeployedJobSpecId() {
@@ -731,6 +734,10 @@
public byte[] getacgBytes() {
return acgBytes;
}
+
+ public boolean getUpsert() {
+ return upsert;
+ }
}
public static class UndeployJobSpecFunction extends CCIdentifiedFunction {
@@ -745,7 +752,7 @@
@Override
public FunctionId getFunctionId() {
- return FunctionId.DESTROY_JOB;
+ return FunctionId.UNDEPLOY_JOB;
}
public DeployedJobSpecId getDeployedJobSpecId() {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
index b78e53f..8242bdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/ipc/NodeControllerRemoteProxy.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.api.job.JobStatus;
import org.apache.hyracks.api.partitions.PartitionId;
import org.apache.hyracks.control.common.base.INodeController;
+import org.apache.hyracks.control.common.controllers.NodeParameters;
import org.apache.hyracks.control.common.ipc.CCNCFunctions.AbortTasksFunction;
import org.apache.hyracks.control.common.ipc.CCNCFunctions.CleanupJobletFunction;
import org.apache.hyracks.control.common.ipc.CCNCFunctions.DeployBinaryFunction;
@@ -50,6 +51,7 @@
import org.apache.hyracks.control.common.ipc.CCNCFunctions.UndeployJobSpecFunction;
import org.apache.hyracks.control.common.job.TaskAttemptDescriptor;
import org.apache.hyracks.ipc.api.IIPCHandle;
+import org.apache.hyracks.ipc.exceptions.IPCException;
public class NodeControllerRemoteProxy implements INodeController {
private final CcId ccId;
@@ -101,8 +103,8 @@
}
@Override
- public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes) throws Exception {
- DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, ccId);
+ public void deployJobSpec(DeployedJobSpecId deployedJobSpecId, byte[] planBytes, boolean upsert) throws Exception {
+ DeployJobSpecFunction fn = new DeployJobSpecFunction(deployedJobSpecId, planBytes, upsert, ccId);
ipcHandle.send(-1, fn, null);
}
@@ -136,6 +138,16 @@
ipcHandle.send(-1, fn, null);
}
+ @Override
+ public void abortJobs(CcId ccId) throws IPCException {
+ ipcHandle.send(-1, new CCNCFunctions.AbortCCJobsFunction(ccId), null);
+ }
+
+ @Override
+ public void sendRegistrationResult(NodeParameters parameters, Exception regFailure) throws IPCException {
+ ipcHandle.send(-1, new CCNCFunctions.NodeRegistrationResult(parameters, regFailure), null);
+ }
+
public InetSocketAddress getAddress() {
return ipcHandle.getRemoteAddress();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
index 735f7cf..08cd5d8 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerIPCI.java
@@ -107,13 +107,13 @@
ncs.getWorkQueue().schedule(new UnDeployBinaryWork(ncs, ndbf.getDeploymentId(), ndbf.getCcId()));
return;
- case DISTRIBUTE_JOB:
+ case DEPLOY_JOB:
CCNCFunctions.DeployJobSpecFunction djf = (CCNCFunctions.DeployJobSpecFunction) fn;
- ncs.getWorkQueue().schedule(
- new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(), djf.getCcId()));
+ ncs.getWorkQueue().schedule(new DeployJobSpecWork(ncs, djf.getDeployedJobSpecId(), djf.getacgBytes(),
+ djf.getUpsert(), djf.getCcId()));
return;
- case DESTROY_JOB:
+ case UNDEPLOY_JOB:
CCNCFunctions.UndeployJobSpecFunction dsjf = (CCNCFunctions.UndeployJobSpecFunction) fn;
ncs.getWorkQueue().schedule(new UndeployJobSpecWork(ncs, dsjf.getDeployedJobSpecId(), dsjf.getCcId()));
return;
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index 6a7d645..aa2320a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -470,7 +470,6 @@
}
private ConcurrentHashMap<CcId, Serializable> getDistributedState() {
- //noinspection unchecked
return (ConcurrentHashMap<CcId, Serializable>) serviceCtx.getDistributedState();
}
@@ -566,9 +565,6 @@
public void storeActivityClusterGraph(DeployedJobSpecId deployedJobSpecId, ActivityClusterGraph acg)
throws HyracksException {
- if (deployedJobSpecActivityClusterGraphMap.get(deployedJobSpecId.getId()) != null) {
- throw HyracksException.create(ErrorCode.DUPLICATE_DEPLOYED_JOB, deployedJobSpecId);
- }
deployedJobSpecActivityClusterGraphMap.put(deployedJobSpecId.getId(), acg);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
index b832b20..6b35912 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/dataset/ResultState.java
@@ -61,9 +61,9 @@
private FileReference fileRef;
- private IFileHandle writeFileHandle;
+ private IFileHandle fileHandle;
- private IFileHandle readFileHandle;
+ private volatile int referenceCount = 0;
private long size;
@@ -86,12 +86,13 @@
localPageList = new ArrayList<>();
fileRef = null;
- writeFileHandle = null;
+ fileHandle = null;
}
public synchronized void open() {
size = 0;
persistentSize = 0;
+ referenceCount = 0;
}
public synchronized void close() {
@@ -112,25 +113,29 @@
}
private void closeWriteFileHandle() {
- if (writeFileHandle != null) {
+ if (fileHandle != null) {
+ doCloseFileHandle();
+ }
+ }
+
+ private void doCloseFileHandle() {
+ if (--referenceCount == 0) {
+ // close the file if there is no more reference
try {
- ioManager.close(writeFileHandle);
+ ioManager.close(fileHandle);
} catch (IOException e) {
// Since file handle could not be closed, just ignore.
}
- writeFileHandle = null;
+ fileHandle = null;
}
}
public synchronized void write(ByteBuffer buffer) throws HyracksDataException {
if (fileRef == null) {
- String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
- fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
- writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_SYNC);
+ initWriteFileHandle();
}
- size += ioManager.syncWrite(writeFileHandle, size, buffer);
+ size += ioManager.syncWrite(fileHandle, size, buffer);
notifyAll();
}
@@ -165,9 +170,8 @@
}
public synchronized void readClose() throws HyracksDataException {
- if (readFileHandle != null) {
- ioManager.close(readFileHandle);
- readFileHandle = null;
+ if (fileHandle != null) {
+ doCloseFileHandle();
}
}
@@ -185,51 +189,49 @@
return readSize;
}
- if (readFileHandle == null) {
+ if (fileHandle == null) {
initReadFileHandle();
}
- readSize = ioManager.syncRead(readFileHandle, offset, buffer);
+ readSize = ioManager.syncRead(fileHandle, offset, buffer);
return readSize;
}
- public long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
+ public synchronized long read(DatasetMemoryManager datasetMemoryManager, long offset, ByteBuffer buffer)
throws HyracksDataException {
long readSize = 0;
- synchronized (this) {
- while (offset >= size && !eos.get() && !failed.get()) {
- try {
- wait();
- } catch (InterruptedException e) {
- throw HyracksDataException.create(e);
- }
+ while (offset >= size && !eos.get() && !failed.get()) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw HyracksDataException.create(e);
}
+ }
- if ((offset >= size && eos.get()) || failed.get()) {
+ if ((offset >= size && eos.get()) || failed.get()) {
+ return readSize;
+ }
+
+ if (offset < persistentSize) {
+ if (fileHandle == null) {
+ initReadFileHandle();
+ }
+ readSize = ioManager.syncRead(fileHandle, offset, buffer);
+ if (readSize < 0) {
+ throw new HyracksDataException("Premature end of file");
+ }
+ }
+
+ if (readSize < buffer.capacity()) {
+ long localPageOffset = offset - persistentSize;
+ int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
+ int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
+ Page page = getPage(localPageIndex);
+ if (page == null) {
return readSize;
}
-
- if (offset < persistentSize) {
- if (readFileHandle == null) {
- initReadFileHandle();
- }
- readSize = ioManager.syncRead(readFileHandle, offset, buffer);
- if (readSize < 0) {
- throw new HyracksDataException("Premature end of file");
- }
- }
-
- if (readSize < buffer.capacity()) {
- long localPageOffset = offset - persistentSize;
- int localPageIndex = (int) (localPageOffset / DatasetMemoryManager.getPageSize());
- int pageOffset = (int) (localPageOffset % DatasetMemoryManager.getPageSize());
- Page page = getPage(localPageIndex);
- if (page == null) {
- return readSize;
- }
- readSize += buffer.remaining();
- buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
- }
+ readSize += buffer.remaining();
+ buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
}
datasetMemoryManager.pageReferenced(resultSetPartitionId);
return readSize;
@@ -245,21 +247,17 @@
// If we do not have any pages to be given back close the write channel since we don't write any more, return null.
if (page == null) {
- ioManager.close(writeFileHandle);
+ ioManager.close(fileHandle);
return null;
}
page.getBuffer().flip();
if (fileRef == null) {
- String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
- fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
- writeFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
- notifyAll();
+ initWriteFileHandle();
}
- long delta = ioManager.syncWrite(writeFileHandle, persistentSize, page.getBuffer());
+ long delta = ioManager.syncWrite(fileHandle, persistentSize, page.getBuffer());
persistentSize += delta;
return page;
}
@@ -325,8 +323,23 @@
return page;
}
+ private void initWriteFileHandle() throws HyracksDataException {
+ if (fileHandle == null) {
+ String fName = FILE_PREFIX + String.valueOf(resultSetPartitionId.getPartition());
+ fileRef = fileFactory.createUnmanagedWorkspaceFile(fName);
+ fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ if (referenceCount != 0) {
+ throw new IllegalStateException("Illegal reference count " + referenceCount);
+ }
+ referenceCount = 1;
+ notifyAll(); // NOSONAR: always called from a synchronized block
+ }
+ }
+
private void initReadFileHandle() throws HyracksDataException {
while (fileRef == null && !failed.get()) {
+ // wait for writer to create the file
try {
wait();
} catch (InterruptedException e) {
@@ -336,9 +349,12 @@
if (failed.get()) {
return;
}
-
- readFileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ if (fileHandle == null) {
+ // fileHandle has been closed by the writer, create it again
+ fileHandle = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_ONLY,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ }
+ referenceCount++;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
index 7de716a..1f845bf 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/io/IOManager.java
@@ -71,7 +71,7 @@
* Mutables
*/
private int workspaceIndex;
- private IFileDeviceResolver deviceComputer;
+ private final IFileDeviceResolver deviceComputer;
public IOManager(List<IODeviceHandle> devices, IFileDeviceResolver deviceComputer) throws HyracksDataException {
this.ioDevices = Collections.unmodifiableList(devices);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
index 33d1d60..6d4f173 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/ApplicationMessageWork.java
@@ -46,7 +46,7 @@
public void run() {
NCServiceContext ctx = ncs.getContext();
try {
- IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);;
+ IMessage data = (IMessage) DeploymentUtils.deserialize(message, deploymentId, ctx);
if (ctx.getMessageBroker() != null) {
ctx.getMessageBroker().receivedMessage(data, nodeId);
} else {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
index 92612dd..bcdb97f 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/work/DeployJobSpecWork.java
@@ -28,7 +28,7 @@
import org.apache.hyracks.control.nc.NodeControllerService;
/**
- * pre-distribute a job that can be executed later
+ * Deploy a job that can be executed later
*
*/
public class DeployJobSpecWork extends AbstractWork {
@@ -37,19 +37,23 @@
private final byte[] acgBytes;
private final CcId ccId;
private final DeployedJobSpecId deployedJobSpecId;
+ private final boolean upsert;
public DeployJobSpecWork(NodeControllerService ncs, DeployedJobSpecId deployedJobSpecId, byte[] acgBytes,
- CcId ccId) {
+ boolean upsert, CcId ccId) {
this.ncs = ncs;
this.deployedJobSpecId = deployedJobSpecId;
this.acgBytes = acgBytes;
this.ccId = ccId;
+ this.upsert = upsert;
}
@Override
public void run() {
try {
- ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+ if (!upsert) {
+ ncs.checkForDuplicateDeployedJobSpec(deployedJobSpecId);
+ }
ActivityClusterGraph acg =
(ActivityClusterGraph) DeploymentUtils.deserialize(acgBytes, null, ncs.getContext());
ncs.storeActivityClusterGraph(deployedJobSpecId, acg);
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 8fdcd83..376307d 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -587,6 +587,31 @@
builder.finish();
}
+ /**
+ * Generates a reversed string from an input source string
+ *
+ * @param srcPtr
+ * , the input source string.
+ * @param builder
+ * , a builder for the resulting string.
+ * @param out
+ * , the storage for a result string.
+ * @throws IOException
+ */
+ public static void reverse(UTF8StringPointable srcPtr, UTF8StringBuilder builder, GrowableArray out)
+ throws IOException {
+ builder.reset(out, srcPtr.getUTF8Length());
+ int srcStart = srcPtr.getCharStartOffset();
+ int srcEnd = srcPtr.getStartOffset() + srcPtr.getLength() - 1;
+ for (int cursorIndex = srcEnd; cursorIndex >= srcStart; cursorIndex--) {
+ if (UTF8StringUtil.isCharStart(srcPtr.bytes, cursorIndex)) {
+ int charSize = UTF8StringUtil.charSize(srcPtr.bytes, cursorIndex);
+ builder.appendUtf8StringPointable(srcPtr, cursorIndex, charSize);
+ }
+ }
+ builder.finish();
+ }
+
public boolean findAndReplace(UTF8StringPointable searchPtr, UTF8StringPointable replacePtr, int replaceLimit,
UTF8StringBuilder builder, GrowableArray out) throws IOException {
return findAndReplace(this, searchPtr, replacePtr, replaceLimit, builder, out);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
index 08ed922..49b5309 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/utils/TupleUtils.java
@@ -164,4 +164,30 @@
tupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
}
}
+
+ public static boolean equalTuples(ITupleReference tuple1, ITupleReference tuple2, int numCmpFields) {
+ for (int i = 0; i < numCmpFields; i++) {
+ if (!equalFields(tuple1, tuple2, i)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ public static boolean equalFields(ITupleReference tuple1, ITupleReference tuple2, int fIdx) {
+ return equalFields(tuple1.getFieldData(fIdx), tuple1.getFieldStart(fIdx), tuple1.getFieldLength(fIdx),
+ tuple2.getFieldData(fIdx), tuple2.getFieldStart(fIdx), tuple2.getFieldLength(fIdx));
+ }
+
+ public static boolean equalFields(byte[] a, int aOffset, int aLength, byte[] b, int bOffset, int bLength) {
+ if (aLength != bLength) {
+ return false;
+ }
+ for (int i = 0; i < aLength; i++) {
+ if (a[aOffset + i] != b[bOffset + i]) {
+ return false;
+ }
+ }
+ return true;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
deleted file mode 100644
index fd16c52..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/IGroupAggregator.java
+++ /dev/null
@@ -1,31 +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.hyracks.dataflow.std.group;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.dataflow.IDataReader;
-import org.apache.hyracks.api.dataflow.IDataWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public interface IGroupAggregator extends Serializable {
- public void aggregate(IDataReader<Object[]> reader, IDataWriter<Object[]> writer) throws HyracksDataException;
-
- public void close() throws Exception;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
index 40b6b27..d8f4064 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/DeployedJobSpecsTest.java
@@ -199,6 +199,16 @@
for (int i = 0; i < 100; i++) {
hcc.startJob(distributedId2, new HashMap<>());
}
+
+ //Change the second job into the first job and see whether it runs
+ hcc.redeployJobSpec(distributedId2, spec1);
+ JobId jobRunId4 = hcc.startJob(distributedId2, new HashMap<>());
+ hcc.waitForCompletion(jobRunId4);
+
+ //Run it one more time
+ JobId jobRunId5 = hcc.startJob(distributedId2, new HashMap<>());
+ hcc.waitForCompletion(jobRunId5);
+
}
@AfterClass
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
index d4f1b3d..891cc2a 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedNettyOutputStream.java
@@ -105,7 +105,7 @@
response.beforeFlush();
DefaultHttpContent content = new DefaultHttpContent(buffer);
ctx.writeAndFlush(content, ctx.channel().voidPromise());
- // The responisbility of releasing the buffer is now with the netty pipeline since it is forwarded
+ // The responsibility of releasing the buffer is now with the netty pipeline since it is forwarded
// within the http content. We must nullify buffer before we allocate the next one to avoid
// releasing the buffer twice in case the allocation call fails.
buffer = null;
@@ -128,13 +128,13 @@
wait();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
- LOGGER.log(Level.WARN, "Interupted while waiting for channel to be writable", e);
+ LOGGER.log(Level.WARN, "Interrupted while waiting for channel to be writable", e);
throw new IOException(e);
}
}
}
- public synchronized void resume() {
+ public synchronized void channelWritabilityChanged() {
notifyAll();
}
}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
index 5a43d25..cd746b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/ChunkedResponse.java
@@ -185,11 +185,11 @@
@Override
public void notifyChannelWritable() {
- outputStream.resume();
+ outputStream.channelWritabilityChanged();
}
@Override
public void notifyChannelInactive() {
- outputStream.resume();
+ outputStream.channelWritabilityChanged();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index ffa4d4b..bdd312d 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -63,6 +63,7 @@
public static class ContentType {
public static final String APPLICATION_ADM = "application/x-adm";
public static final String APPLICATION_JSON = "application/json";
+ public static final String JSON = "json";
public static final String APPLICATION_X_WWW_FORM_URLENCODED = "application/x-www-form-urlencoded";
public static final String CSV = "text/csv";
public static final String IMG_PNG = "image/png";
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
index 7f59db1..1f9ec37 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/pom.xml
@@ -41,10 +41,6 @@
</properties>
<dependencies>
<dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- </dependency>
- <dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
deleted file mode 100644
index 4328fc3..0000000
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/api/IResponseCallback.java
+++ /dev/null
@@ -1,23 +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.hyracks.ipc.api;
-
-public interface IResponseCallback {
- public void callback(IIPCHandle handle, Object response, Exception exception);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
index 912c267..39a29af 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/HandleState.java
@@ -23,6 +23,5 @@
CONNECT_SENT,
CONNECT_RECEIVED,
CONNECTED,
- CONNECT_FAILED,
CLOSED,
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index 3e6c64b..040fe03 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -18,12 +18,13 @@
*/
package org.apache.hyracks.ipc.impl;
+import static org.apache.hyracks.util.ExitUtil.EC_IMMEDIATE_HALT;
+
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.nio.ByteBuffer;
import java.nio.channels.Channel;
-import java.nio.channels.ClosedChannelException;
import java.nio.channels.SelectableChannel;
import java.nio.channels.SelectionKey;
import java.nio.channels.Selector;
@@ -39,7 +40,7 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;
-import org.apache.commons.io.IOUtils;
+import org.apache.hyracks.util.ExitUtil;
import org.apache.hyracks.util.NetworkUtil;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -100,7 +101,7 @@
void stop() {
stopped = true;
- IOUtils.closeQuietly(serverSocketChannel);
+ NetworkUtil.closeQuietly(serverSocketChannel);
networkThread.selector.wakeup();
}
@@ -121,8 +122,10 @@
return handle;
}
if (maxRetries < 0 || retries++ < maxRetries) {
- LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
- : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+ if (LOGGER.isWarnEnabled()) {
+ LOGGER.warn("Connection to " + remoteAddress + " failed; retrying" + (maxRetries <= 0 ? ""
+ : " (retry attempt " + retries + " of " + maxRetries + ") after " + delay + "ms"));
+ }
Thread.sleep(delay);
delay = Math.min(MAX_RETRY_DELAY_MILLIS, (int) (delay * 1.5));
} else {
@@ -144,24 +147,6 @@
networkThread.selector.wakeup();
}
- private synchronized void collectOutstandingWork() {
- if (!pendingConnections.isEmpty()) {
- moveAll(pendingConnections, workingPendingConnections);
- }
- if (!sendList.isEmpty()) {
- moveAll(sendList, workingSendList);
- }
- }
-
- private Message createInitialReqMessage(IPCHandle handle) {
- Message msg = new Message(handle);
- msg.setMessageId(system.createMessageId());
- msg.setRequestMessageId(-1);
- msg.setFlag(Message.INITIAL_REQ);
- msg.setPayload(address);
- return msg;
- }
-
private Message createInitialAckMessage(IPCHandle handle, Message req) {
Message msg = new Message(handle);
msg.setMessageId(system.createMessageId());
@@ -177,16 +162,18 @@
private class NetworkThread extends Thread {
private final Selector selector;
-
private final Set<SocketChannel> openChannels = new HashSet<>();
+ private final BitSet unsentMessagesBitmap = new BitSet();
+ private final List<Message> tempUnsentMessages = new ArrayList<>();
- public NetworkThread() {
+ NetworkThread() {
super("IPC Network Listener Thread [" + address + "]");
setDaemon(true);
try {
selector = Selector.open();
+ serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
} catch (IOException e) {
- throw new RuntimeException(e);
+ throw new IllegalStateException(e);
}
}
@@ -200,105 +187,19 @@
}
private void doRun() {
- try {
- serverSocketChannel.register(selector, SelectionKey.OP_ACCEPT);
- } catch (ClosedChannelException e) {
- throw new RuntimeException(e);
- }
- BitSet unsentMessagesBitmap = new BitSet();
- List<Message> tempUnsentMessages = new ArrayList<>();
int failingLoops = 0;
while (!stopped) {
try {
- if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("Starting Select");
- }
int n = selector.select();
collectOutstandingWork();
if (!workingPendingConnections.isEmpty()) {
- for (IPCHandle handle : workingPendingConnections) {
- SocketChannel channel = SocketChannel.open();
- register(channel);
- SelectionKey cKey;
- if (channel.connect(handle.getRemoteAddress())) {
- cKey = channel.register(selector, SelectionKey.OP_READ);
- handle.setState(HandleState.CONNECT_SENT);
- IPCConnectionManager.this.write(createInitialReqMessage(handle));
- } else {
- cKey = channel.register(selector, SelectionKey.OP_CONNECT);
- }
- handle.setKey(cKey);
- cKey.attach(handle);
- }
- workingPendingConnections.clear();
+ establishPendingConnections();
}
if (!workingSendList.isEmpty()) {
- unsentMessagesBitmap.clear();
- int len = workingSendList.size();
- for (int i = 0; i < len; ++i) {
- Message msg = workingSendList.get(i);
- LOGGER.debug(() -> "Processing send of message: " + msg);
- IPCHandle handle = msg.getIPCHandle();
- if (handle.getState() != HandleState.CLOSED) {
- if (!handle.full()) {
- while (true) {
- ByteBuffer buffer = handle.getOutBuffer();
- buffer.compact();
- boolean success = msg.write(buffer);
- buffer.flip();
- if (success) {
- system.getPerformanceCounters().addMessageSentCount(1);
- SelectionKey key = handle.getKey();
- key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
- } else {
- if (!buffer.hasRemaining()) {
- handle.resizeOutBuffer();
- continue;
- }
- handle.markFull();
- unsentMessagesBitmap.set(i);
- }
- break;
- }
- } else {
- unsentMessagesBitmap.set(i);
- }
- }
- }
- copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+ sendPendingMessages();
}
if (n > 0) {
- for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
- SelectionKey key = i.next();
- i.remove();
- final SelectableChannel sc = key.channel();
- if (key.isReadable()) {
- read(key);
- } else if (key.isWritable()) {
- write(key);
- } else if (key.isAcceptable()) {
- assert sc == serverSocketChannel;
- SocketChannel channel = serverSocketChannel.accept();
- register(channel);
- IPCHandle handle = new IPCHandle(system, null);
- SelectionKey cKey = channel.register(selector, SelectionKey.OP_READ);
- handle.setKey(cKey);
- cKey.attach(handle);
- handle.setState(HandleState.CONNECT_RECEIVED);
- } else if (key.isConnectable()) {
- SocketChannel channel = (SocketChannel) sc;
- IPCHandle handle = (IPCHandle) key.attachment();
- if (!finishConnect(channel)) {
- handle.setState(HandleState.CONNECT_FAILED);
- continue;
- }
-
- handle.setState(HandleState.CONNECT_SENT);
- registerHandle(handle);
- key.interestOps(SelectionKey.OP_READ);
- IPCConnectionManager.this.write(createInitialReqMessage(handle));
- }
- }
+ processSelectedKeys();
}
// reset failingLoops on a good loop
failingLoops = 0;
@@ -314,25 +215,146 @@
}
}
- private void cleanup() {
- for (Channel channel : openChannels) {
- IOUtils.closeQuietly(channel);
+ private void processSelectedKeys() {
+ for (Iterator<SelectionKey> i = selector.selectedKeys().iterator(); i.hasNext();) {
+ SelectionKey key = i.next();
+ i.remove();
+ final SelectableChannel sc = key.channel();
+ if (key.isReadable()) {
+ read(key);
+ } else if (key.isWritable()) {
+ write(key);
+ } else if (key.isAcceptable()) {
+ assert sc == serverSocketChannel;
+ accept();
+ } else if (key.isConnectable()) {
+ finishConnect(key);
+ }
}
- openChannels.clear();
- IOUtils.closeQuietly(selector);
}
- private boolean finishConnect(SocketChannel channel) {
- boolean connectFinished = false;
+ private void finishConnect(SelectionKey connectableKey) {
+ SocketChannel channel = (SocketChannel) connectableKey.channel();
+ IPCHandle handle = (IPCHandle) connectableKey.attachment();
+ boolean connected = false;
try {
- connectFinished = channel.finishConnect();
- if (!connectFinished) {
- LOGGER.log(Level.WARN, "Channel connect did not finish");
+ connected = channel.finishConnect();
+ if (connected) {
+ connectableKey.interestOps(SelectionKey.OP_READ);
+ connectionEstablished(handle);
}
} catch (IOException e) {
- LOGGER.log(Level.WARN, "Exception finishing channel connect", e);
+ LOGGER.warn("Exception finishing connect", e);
+ } finally {
+ if (!connected) {
+ LOGGER.warn("Failed to finish connect to {}", handle.getRemoteAddress());
+ close(connectableKey, channel);
+ }
}
- return connectFinished;
+ }
+
+ private void accept() {
+ SocketChannel channel = null;
+ SelectionKey channelKey = null;
+ try {
+ channel = serverSocketChannel.accept();
+ register(channel);
+ channelKey = channel.register(selector, SelectionKey.OP_READ);
+ IPCHandle handle = new IPCHandle(system, null);
+ handle.setKey(channelKey);
+ channelKey.attach(handle);
+ handle.setState(HandleState.CONNECT_RECEIVED);
+ } catch (IOException e) {
+ LOGGER.error("Failed to accept channel ", e);
+ close(channelKey, channel);
+ }
+ }
+
+ private void establishPendingConnections() {
+ for (IPCHandle handle : workingPendingConnections) {
+ SocketChannel channel = null;
+ SelectionKey channelKey = null;
+ try {
+ channel = SocketChannel.open();
+ register(channel);
+ if (channel.connect(handle.getRemoteAddress())) {
+ channelKey = channel.register(selector, SelectionKey.OP_READ);
+ connectionEstablished(handle);
+ } else {
+ channelKey = channel.register(selector, SelectionKey.OP_CONNECT);
+ }
+ handle.setKey(channelKey);
+ channelKey.attach(handle);
+ } catch (IOException e) {
+ LOGGER.error("Failed to accept channel ", e);
+ close(channelKey, channel);
+ }
+ }
+ workingPendingConnections.clear();
+ }
+
+ private void connectionEstablished(IPCHandle handle) {
+ handle.setState(HandleState.CONNECT_SENT);
+ registerHandle(handle);
+ IPCConnectionManager.this.write(createInitialReqMessage(handle));
+ }
+
+ private void sendPendingMessages() {
+ unsentMessagesBitmap.clear();
+ int len = workingSendList.size();
+ for (int i = 0; i < len; ++i) {
+ Message msg = workingSendList.get(i);
+ final boolean sent = sendMessage(msg);
+ if (!sent) {
+ unsentMessagesBitmap.set(i);
+ }
+ }
+ copyUnsentMessages(unsentMessagesBitmap, tempUnsentMessages);
+ }
+
+ private boolean sendMessage(Message msg) {
+ LOGGER.debug("Processing send of message: {}", msg);
+ IPCHandle handle = msg.getIPCHandle();
+ if (handle.getState() == HandleState.CLOSED) {
+ // message will never be sent
+ return true;
+ }
+ if (handle.full()) {
+ return false;
+ }
+ try {
+ while (true) {
+ ByteBuffer buffer = handle.getOutBuffer();
+ buffer.compact();
+ boolean success = msg.write(buffer);
+ buffer.flip();
+ if (success) {
+ system.getPerformanceCounters().addMessageSentCount(1);
+ SelectionKey key = handle.getKey();
+ key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+ return true;
+ } else {
+ if (!buffer.hasRemaining()) {
+ handle.resizeOutBuffer();
+ continue;
+ }
+ handle.markFull();
+ return false;
+ }
+ }
+ } catch (Exception e) {
+ LOGGER.fatal("Unrecoverable networking failure; Halting...", e);
+ ExitUtil.halt(EC_IMMEDIATE_HALT);
+ }
+ return false;
+ }
+
+ private void cleanup() {
+ for (Channel channel : openChannels) {
+ NetworkUtil.closeQuietly(channel);
+ }
+ openChannels.clear();
+ NetworkUtil.closeQuietly(selector);
}
private void copyUnsentMessages(BitSet unsentMessagesBitmap, List<Message> tempUnsentMessages) {
@@ -396,19 +418,42 @@
}
private void close(SelectionKey key, SocketChannel sc) {
- key.cancel();
- NetworkUtil.closeQuietly(sc);
- openChannels.remove(sc);
- final IPCHandle handle = (IPCHandle) key.attachment();
- handle.close();
+ if (key != null) {
+ final Object attachment = key.attachment();
+ if (attachment != null) {
+ ((IPCHandle) attachment).close();
+ }
+ key.cancel();
+ }
+ if (sc != null) {
+ NetworkUtil.closeQuietly(sc);
+ openChannels.remove(sc);
+ }
}
- }
- private <T> void moveAll(List<T> source, List<T> target) {
- int len = source.size();
- for (int i = 0; i < len; ++i) {
- target.add(source.get(i));
+ private void collectOutstandingWork() {
+ synchronized (IPCConnectionManager.this) {
+ if (!pendingConnections.isEmpty()) {
+ moveAll(pendingConnections, workingPendingConnections);
+ }
+ if (!sendList.isEmpty()) {
+ moveAll(sendList, workingSendList);
+ }
+ }
}
- source.clear();
+
+ private Message createInitialReqMessage(IPCHandle handle) {
+ Message msg = new Message(handle);
+ msg.setMessageId(system.createMessageId());
+ msg.setRequestMessageId(-1);
+ msg.setFlag(Message.INITIAL_REQ);
+ msg.setPayload(address);
+ return msg;
+ }
+
+ private <T> void moveAll(List<T> source, List<T> target) {
+ target.addAll(source);
+ source.clear();
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
index d63bfbd..09c7c97 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCHandle.java
@@ -62,10 +62,6 @@
return system;
}
- void setRemoteAddress(InetSocketAddress remoteAddress) {
- this.remoteAddress = remoteAddress;
- }
-
@Override
public long send(long requestId, Object req, Exception exception) throws IPCException {
if (!isConnected()) {
@@ -127,7 +123,6 @@
wait();
break;
case CONNECTED:
- case CONNECT_FAILED:
case CLOSED:
return state == HandleState.CONNECTED;
default:
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
index b36e645..b7dcf05 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCSystem.java
@@ -66,10 +66,6 @@
cMgr.stop();
}
- public IIPCHandle getHandle(InetSocketAddress remoteAddress) throws IPCException {
- return getHandle(remoteAddress, 0);
- }
-
public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries) throws IPCException {
return getHandle(remoteAddress, maxRetries, 0);
}
@@ -78,16 +74,6 @@
return getReconnectingHandle(remoteAddress, 1);
}
- public IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
- throws IPCException {
- return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
- }
-
- public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
- throws IPCException {
- return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
- }
-
public IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts,
IIPCEventListener eventListener) throws IPCException {
if (reconnectAttempts > 0) {
@@ -132,4 +118,14 @@
public IPCPerformanceCounters getPerformanceCounters() {
return perfCounters;
}
+
+ private IIPCHandle getReconnectingHandle(InetSocketAddress remoteAddress, int reconnectAttempts)
+ throws IPCException {
+ return getHandle(remoteAddress, 0, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+ }
+
+ private IIPCHandle getHandle(InetSocketAddress remoteAddress, int maxRetries, int reconnectAttempts)
+ throws IPCException {
+ return getHandle(remoteAddress, maxRetries, reconnectAttempts, NoOpIPCEventListener.INSTANCE);
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
index cca3abe..70a0e18 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/test/java/org/apache/hyracks/ipc/tests/IPCTest.java
@@ -43,7 +43,7 @@
IPCSystem client = createClientIPCSystem(rpci);
client.start();
- IIPCHandle handle = client.getHandle(serverAddr);
+ IIPCHandle handle = client.getHandle(serverAddr, 0);
for (int i = 0; i < 100; ++i) {
Assert.assertEquals(rpci.call(handle, Integer.valueOf(i)), Integer.valueOf(2 * i));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 49eca6f..566d8e2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -27,6 +27,7 @@
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
public class BTreeSearchOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
@@ -48,6 +49,8 @@
protected boolean appendOpCallbackProceedResult;
protected byte[] searchCallbackProceedResultFalseValue;
protected byte[] searchCallbackProceedResultTrueValue;
+ protected final ITupleFilterFactory tupleFilterFactory;
+ protected final long outputLimit;
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive,
@@ -56,7 +59,7 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter) {
this(spec, outRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, minFilterFieldIndexes,
- maxFilterFieldIndexes, appendIndexFilter, false, null, null);
+ maxFilterFieldIndexes, appendIndexFilter, null, -1, false, null, null);
}
public BTreeSearchOperatorDescriptor(IOperatorDescriptorRegistry spec, RecordDescriptor outRecDesc,
@@ -64,8 +67,8 @@
IIndexDataflowHelperFactory indexHelperFactory, boolean retainInput, boolean retainMissing,
IMissingWriterFactory missingWriterFactory, ISearchOperationCallbackFactory searchCallbackFactory,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, boolean appendIndexFilter,
- boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+ byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue) {
super(spec, 1, 1);
this.indexHelperFactory = indexHelperFactory;
this.retainInput = retainInput;
@@ -80,6 +83,8 @@
this.maxFilterFieldIndexes = maxFilterFieldIndexes;
this.appendIndexFilter = appendIndexFilter;
this.outRecDescs[0] = outRecDesc;
+ this.tupleFilterFactory = tupleFilterFactory;
+ this.outputLimit = outputLimit;
this.appendOpCallbackProceedResult = appendOpCallbackProceedResult;
this.searchCallbackProceedResultFalseValue = searchCallbackProceedResultFalseValue;
this.searchCallbackProceedResultTrueValue = searchCallbackProceedResultTrueValue;
@@ -92,7 +97,7 @@
recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), lowKeyFields, highKeyFields,
lowKeyInclusive, highKeyInclusive, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+ tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
searchCallbackProceedResultTrueValue);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 6bbf437..70b88fe 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -26,6 +26,7 @@
import org.apache.hyracks.storage.am.btree.util.BTreeUtils;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
import org.apache.hyracks.storage.am.common.dataflow.IndexSearchOperatorNodePushable;
import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -50,7 +51,7 @@
throws HyracksDataException {
this(ctx, partition, inputRecDesc, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory, retainInput, retainMissing,
- missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null, null);
+ missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1, false, null, null);
}
public BTreeSearchOperatorNodePushable(IHyracksTaskContext ctx, int partition, RecordDescriptor inputRecDesc,
@@ -58,11 +59,12 @@
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- boolean appendOpCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit, boolean appendOpCallbackProceedResult,
+ byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+ throws HyracksDataException {
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
- appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
+ tupleFilterFactory, outputLimit, appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
searchCallbackProceedResultTrueValue);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
index 50673cf..b5dfe00 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITupleFilter.java
@@ -19,8 +19,9 @@
package org.apache.hyracks.storage.am.common.api;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
public interface ITupleFilter {
- public boolean accept(IFrameTupleReference tuple) throws Exception;
+ public boolean accept(IFrameTupleReference tuple) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
index 16c0afa..c7be7eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITwoPCIndexBulkLoader.java
@@ -39,6 +39,6 @@
/**
* Abort the bulk modify op
*/
- public void abort();
+ public void abort() throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index 2c691f1..fb287cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -22,6 +22,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
@@ -37,11 +38,14 @@
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import org.apache.hyracks.storage.am.common.api.IIndexDataflowHelper;
import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+import org.apache.hyracks.storage.am.common.api.ITupleFilter;
+import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.tuples.PermutingFrameTupleReference;
@@ -95,22 +99,43 @@
protected byte[] searchCallbackProceedResultFalseValue;
protected byte[] searchCallbackProceedResultTrueValue;
+ protected final ITupleFilterFactory tupleFilterFactory;
+ protected ReferenceFrameTupleReference referenceFilterTuple;
+ // filter out tuples based on the query-provided select condition
+ // only results satisfying the filter condition would be returned to downstream operators
+ protected ITupleFilter tupleFilter;
+ protected final long outputLimit;
+ protected long outputCount = 0;
+ protected boolean finished;
+
+ // no filter and limit pushdown
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter)
throws HyracksDataException {
this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, false, null,
- null);
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
+ false, null, null);
}
public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
- boolean appendSearchCallbackProceedResult, byte[] searchCallbackProceedResultFalseValue,
- byte[] searchCallbackProceedResultTrueValue) throws HyracksDataException {
+ ITupleFilterFactory tupleFilterFactory, long outputLimit) throws HyracksDataException {
+ this(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ tupleFilterFactory, outputLimit, false, null, null);
+ }
+
+ public IndexSearchOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecDesc, int partition,
+ int[] minFilterFieldIndexes, int[] maxFilterFieldIndexes, IIndexDataflowHelperFactory indexHelperFactory,
+ boolean retainInput, boolean retainMissing, IMissingWriterFactory missingWriterFactory,
+ ISearchOperationCallbackFactory searchCallbackFactory, boolean appendIndexFilter,
+ ITupleFilterFactory tupleFactoryFactory, long outputLimit, boolean appendSearchCallbackProceedResult,
+ byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
+ throws HyracksDataException {
this.ctx = ctx;
this.indexHelper = indexHelperFactory.create(ctx.getJobletContext().getServiceContext(), partition);
this.retainInput = retainInput;
@@ -138,6 +163,12 @@
if (ctx.getStatsCollector() != null) {
ctx.getStatsCollector().add(stats);
}
+ this.tupleFilterFactory = tupleFactoryFactory;
+ this.outputLimit = outputLimit;
+
+ if (this.tupleFilterFactory != null && this.retainMissing) {
+ throw new IllegalStateException("RetainMissing with tuple filter is not supported");
+ }
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -179,6 +210,13 @@
buildMissingTuple(numIndexFilterFields, nonFilterTupleBuild, nonMatchWriter);
}
+ if (tupleFilterFactory != null) {
+ tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
+ referenceFilterTuple = new ReferenceFrameTupleReference();
+ }
+ finished = false;
+ outputCount = 0;
+
try {
searchPred = createSearchPredicate();
tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
@@ -201,9 +239,14 @@
protected void writeSearchResults(int tupleIndex) throws Exception {
long matchingTupleCount = 0;
while (cursor.hasNext()) {
- matchingTupleCount++;
- tb.reset();
cursor.next();
+ matchingTupleCount++;
+ ITupleReference tuple = cursor.getTuple();
+ if (tupleFilter != null && !tupleFilter.accept(referenceFilterTuple.reset(tuple))) {
+ continue;
+ }
+ tb.reset();
+
if (retainInput) {
frameTuple.reset(accessor, tupleIndex);
for (int i = 0; i < frameTuple.getFieldCount(); i++) {
@@ -211,7 +254,6 @@
tb.addFieldEndOffset();
}
}
- ITupleReference tuple = cursor.getTuple();
writeTupleToOutput(tuple);
if (appendSearchCallbackProceedResult) {
writeSearchCallbackProceedResult(tb,
@@ -222,6 +264,10 @@
writeFilterTupleToOutput(((ILSMIndexCursor) cursor).getFilterMaxTuple());
}
FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize());
+ if (outputLimit >= 0 && ++outputCount >= outputLimit) {
+ finished = true;
+ break;
+ }
}
stats.getTupleCounter().update(matchingTupleCount);
@@ -237,7 +283,7 @@
accessor.reset(buffer);
int tupleCount = accessor.getTupleCount();
try {
- for (int i = 0; i < tupleCount; i++) {
+ for (int i = 0; i < tupleCount && !finished; i++) {
resetSearchPredicate(i);
cursor.close();
indexAccessor.search(cursor, searchPred);
@@ -344,4 +390,50 @@
}
}
+ /**
+ * A wrapper class to wrap ITupleReference into IFrameTupleReference, as the latter
+ * is used by ITupleFilter
+ *
+ */
+ private static class ReferenceFrameTupleReference implements IFrameTupleReference {
+ private ITupleReference tuple;
+
+ public IFrameTupleReference reset(ITupleReference tuple) {
+ this.tuple = tuple;
+ return this;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return tuple.getFieldCount();
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ return tuple.getFieldData(fIdx);
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ return tuple.getFieldStart(fIdx);
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ return tuple.getFieldLength(fIdx);
+ }
+
+ @Override
+ public IFrameTupleAccessor getFrameTupleAccessor() {
+ throw new UnsupportedOperationException(
+ "getFrameTupleAccessor is not supported by ReferenceFrameTupleReference");
+ }
+
+ @Override
+ public int getTupleIndex() {
+ throw new UnsupportedOperationException("getTupleIndex is not supported by ReferenceFrameTupleReference");
+ }
+
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
index 636e4f5..56600b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/IndexOperation.java
@@ -35,6 +35,5 @@
FLUSH,
REPLICATE,
DISK_COMPONENT_SCAN,
- DELETE_MEMORY_COMPONENT,
- DELETE_DISK_COMPONENTS
+ DELETE_COMPONENTS
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
index 18de540..c635411 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/tuples/ConcatenatingTupleReference.java
@@ -19,8 +19,6 @@
package org.apache.hyracks.storage.am.common.tuples;
-import java.util.Arrays;
-
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
/**
@@ -96,14 +94,17 @@
return tuples[tupleIndex].getFieldLength(fieldIndex);
}
+ /**
+ * Right now this class is only used by inverted index, and only contains 2 tuples.
+ * As a result, sequential search would be more efficient than binary search
+ */
private int getTupleIndex(int fIdx) {
- int tupleIndex = Arrays.binarySearch(fieldCounts, 0, numTuples - 1, fIdx);
- if (tupleIndex < 0) {
- tupleIndex = -tupleIndex - 1;
- } else {
- ++tupleIndex;
+ for (int i = 0; i < numTuples; i++) {
+ if (fIdx < fieldCounts[i]) {
+ return i;
+ }
}
- return tupleIndex;
+ throw new IllegalArgumentException("Illegal field index " + fIdx);
}
private int getFieldIndex(int tupleIndex, int fIdx) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
index c0f7571..a504f7e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTree.java
@@ -22,6 +22,7 @@
import java.util.Collections;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -40,8 +41,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -54,6 +53,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor.ICursorFactory;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
@@ -170,8 +170,7 @@
// The only reason to override the following method is that it uses a different context object
// in addition, determining whether or not to keep deleted tuples is different here
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
+ public LSMBTreeMergeOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
opCtx.setOperation(IndexOperation.MERGE);
List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -195,9 +194,11 @@
LSMComponentFileReferences relMergeFileRefs =
fileManager.getRelMergeFileReference(firstFile.getFile().getName(), lastFile.getFile().getName());
ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
- ioScheduler.scheduleOperation(new LSMBTreeMergeOperation(accessor, cursor,
+ LSMBTreeMergeOperation mergeOp = new LSMBTreeMergeOperation(accessor, cursor,
relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
- callback, fileManager.getBaseDir().getAbsolutePath()));
+ ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+ ioOpCallback.scheduled(mergeOp);
+ return mergeOp;
}
// This function should only be used when a transaction fail. it doesn't
@@ -286,11 +287,6 @@
if (!isActive) {
throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
- if (flushOnExit) {
- ExternalBTreeOpContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
- opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- ioOpCallback.afterFinalize(opCtx);
- }
for (ILSMDiskComponent c : diskComponents) {
c.deactivateAndPurge();
}
@@ -353,8 +349,7 @@
// Not supported
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
+ public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-BTree");
}
@@ -401,16 +396,16 @@
// For initial load
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false);
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, false, parameters);
}
// For transaction bulk load <- could consolidate with the above method ->
@Override
- public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+ public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCBTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
}
// The bulk loader used for both initial loading and transaction
@@ -418,21 +413,37 @@
public class LSMTwoPCBTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
private final ILSMDiskComponent component;
private final ILSMDiskComponentBulkLoader componentBulkLoader;
+ private final LoadOperation loadOp;
private final boolean isTransaction;
public LSMTwoPCBTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean isTransaction) throws HyracksDataException {
+ boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
this.isTransaction = isTransaction;
// Create the appropriate target
+ LSMComponentFileReferences componentFileRefs;
if (isTransaction) {
- component = createTransactionTarget();
+ try {
+ componentFileRefs = fileManager.getNewTransactionFileReference();
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ component = createDiskComponent(transactionComponentFactory,
+ componentFileRefs.getInsertIndexFileReference(), null,
+ componentFileRefs.getBloomFilterFileReference(), true);
} else {
- component = createBulkLoadTarget();
+ componentFileRefs = fileManager.getRelFlushFileReference();
+ component =
+ createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+ componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true);
}
-
+ loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+ loadOp.setNewComponent(component);
+ ioOpCallback.scheduled(loadOp);
+ ioOpCallback.beforeOperation(loadOp);
componentBulkLoader =
- component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+ component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, true);
}
// It is expected that the mode was set to insert operation before
@@ -444,16 +455,23 @@
@Override
public void end() throws HyracksDataException {
- componentBulkLoader.end();
- if (component.getComponentSize() > 0) {
- if (isTransaction) {
- // Since this is a transaction component, validate and
- // deactivate. it could later be added or deleted
- component.markAsValid(durable);
- component.deactivate();
- } else {
- getHarness().addBulkLoadedComponent(component);
+ try {
+ ioOpCallback.afterOperation(loadOp);
+ componentBulkLoader.end();
+ if (component.getComponentSize() > 0) {
+ if (isTransaction) {
+ // Since this is a transaction component, validate and
+ // deactivate. it could later be added or deleted
+ component.markAsValid(durable);
+ ioOpCallback.afterFinalize(loadOp);
+ component.deactivate();
+ } else {
+ ioOpCallback.afterFinalize(loadOp);
+ getHarness().addBulkLoadedComponent(component);
+ }
}
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
@@ -465,26 +483,13 @@
}
@Override
- public void abort() {
+ public void abort() throws HyracksDataException {
try {
componentBulkLoader.abort();
- } catch (Exception e) {
- // Do nothing
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
-
- // This method is used to create a target for a bulk modify operation. This
- // component must then be either committed or deleted
- private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
- LSMComponentFileReferences componentFileRefs;
- try {
- componentFileRefs = fileManager.getNewTransactionFileReference();
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
- return createDiskComponent(transactionComponentFactory, componentFileRefs.getInsertIndexFileReference(),
- null, componentFileRefs.getBloomFilterFileReference(), true);
- }
}
// The accessor for disk only indexes don't use modification callback and always carry the target index version with them
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
index 1ba55f7..b727a39 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/ExternalBTreeWithBuddy.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -35,7 +36,6 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndex;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBloomFilterDiskComponent;
@@ -44,7 +44,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -59,6 +58,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMTreeIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
@@ -217,16 +217,16 @@
// For initial load
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false);
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, 0, false, parameters);
}
// For transaction bulk load <- could consolidate with the above method ->
@Override
- public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+ public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCBTreeWithBuddyBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
}
@Override
@@ -244,8 +244,7 @@
}
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
+ public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
throw HyracksDataException.create(ErrorCode.FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX);
}
@@ -267,7 +266,7 @@
}
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
+ public LSMBTreeWithBuddyMergeOperation createMergeOperation(ILSMIndexOperationContext ctx)
throws HyracksDataException {
ILSMIndexOperationContext bctx = createOpContext(NoOpOperationCallback.INSTANCE, 0);
bctx.setOperation(IndexOperation.MERGE);
@@ -289,10 +288,13 @@
.get(secondDiskComponents.size() - 1);
}
- ioScheduler.scheduleOperation(
+ LSMBTreeWithBuddyMergeOperation mergeOp =
new LSMBTreeWithBuddyMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
- callback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples));
+ ioOpCallback, fileManager.getBaseDir().getAbsolutePath(), keepDeleteTuples);
+ ioOpCallback.scheduled(mergeOp);
+ return mergeOp;
+
}
// This method creates the appropriate opContext for the targeted version
@@ -331,7 +333,8 @@
numElements += ((AbstractLSMWithBloomFilterDiskComponent) mergeOp.getMergingComponents().get(i))
.getBloomFilter().getNumElements();
}
- componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+ componentBulkLoader =
+ mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
try {
while (buddyBtreeCursor.hasNext()) {
buddyBtreeCursor.next();
@@ -342,7 +345,7 @@
buddyBtreeCursor.close();
}
} else {
- componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+ componentBulkLoader = mergedComponent.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
}
try {
@@ -430,11 +433,6 @@
if (!isActive) {
throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
- if (flushOnExit) {
- AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
- opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- ioOpCallback.afterFinalize(opCtx);
- }
// Even though, we deactivate the index, we don't exit components or
// modify any of the lists to make sure they
// are there if the index was opened again
@@ -497,21 +495,39 @@
// modifications
public class LSMTwoPCBTreeWithBuddyBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
private final ILSMDiskComponent component;
+ private final LoadOperation loadOp;
private final ILSMDiskComponentBulkLoader componentBulkLoader;
private final boolean isTransaction;
public LSMTwoPCBTreeWithBuddyBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean isTransaction) throws HyracksDataException {
+ boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
this.isTransaction = isTransaction;
// Create the appropriate target
+ LSMComponentFileReferences componentFileRefs;
if (isTransaction) {
- component = createTransactionTarget();
+ try {
+ componentFileRefs = fileManager.getNewTransactionFileReference();
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ component =
+ createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+ componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true);
} else {
- component = createBulkLoadTarget();
+ componentFileRefs = fileManager.getRelFlushFileReference();
+ component =
+ createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+ componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true);
}
+ loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+ loadOp.setNewComponent(component);
+ ioOpCallback.scheduled(loadOp);
+ ioOpCallback.beforeOperation(loadOp);
componentBulkLoader =
- component.createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, false);
+ component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
}
@Override
@@ -521,16 +537,23 @@
@Override
public void end() throws HyracksDataException {
- componentBulkLoader.end();
- if (component.getComponentSize() > 0) {
- if (isTransaction) {
- // Since this is a transaction component, validate and
- // deactivate. it could later be added or deleted
- component.markAsValid(durable);
- component.deactivate();
- } else {
- getHarness().addBulkLoadedComponent(component);
+ try {
+ ioOpCallback.afterOperation(loadOp);
+ componentBulkLoader.end();
+ if (component.getComponentSize() > 0) {
+ if (isTransaction) {
+ // Since this is a transaction component, validate and
+ // deactivate. it could later be added or deleted
+ component.markAsValid(durable);
+ ioOpCallback.afterFinalize(loadOp);
+ component.deactivate();
+ } else {
+ ioOpCallback.afterFinalize(loadOp);
+ getHarness().addBulkLoadedComponent(component);
+ }
}
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
@@ -540,26 +563,17 @@
}
@Override
- public void abort() {
+ public void abort() throws HyracksDataException {
try {
- componentBulkLoader.abort();
- } catch (Exception e) {
+ try {
+ componentBulkLoader.abort();
+ } finally {
+ ioOpCallback.afterFinalize(loadOp);
+ }
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
-
- // This method is used to create a target for a bulk modify operation. This
- // component must then eventually be either committed or deleted
- private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
- LSMComponentFileReferences componentFileRefs;
- try {
- componentFileRefs = fileManager.getNewTransactionFileReference();
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
- return createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
- componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
- true);
- }
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
index 41a11e6..5332b1e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTree.java
@@ -74,7 +74,7 @@
public class LSMBTree extends AbstractLSMIndex implements ITreeIndex {
- private static final ICursorFactory cursorFactory = opCtx -> new LSMBTreeSearchCursor(opCtx);
+ private static final ICursorFactory cursorFactory = LSMBTreeSearchCursor::new;
// Common for in-memory and on-disk components.
protected final ITreeIndexFrameFactory insertLeafFrameFactory;
protected final ITreeIndexFrameFactory deleteLeafFrameFactory;
@@ -109,8 +109,7 @@
new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache), interiorFrameFactory,
insertLeafFrameFactory, cmpFactories, fieldCount,
ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_" + i)),
- virtualBufferCache, i == 0 ? true : false,
- filterHelper == null ? null : filterHelper.createFilter());
+ virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
memoryComponents.add(mutableComponent);
++i;
}
@@ -277,7 +276,7 @@
}
component = createDiskComponent(componentFactory, flushOp.getTarget(), null, flushOp.getBloomFilterTarget(),
true);
- componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
IIndexCursor scanCursor = accessor.createSearchCursor(false);
accessor.search(scanCursor, nullPred);
try {
@@ -337,7 +336,7 @@
mergedComponent = createDiskComponent(componentFactory, mergeOp.getTarget(), null,
mergeOp.getBloomFilterTarget(), true);
componentBulkLoader =
- mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+ mergedComponent.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
while (cursor.hasNext()) {
cursor.next();
ITupleReference frameTuple = cursor.getTuple();
@@ -391,7 +390,7 @@
LSMComponentFileReferences componentFileRefs, ILSMIOOperationCallback callback) {
ILSMIndexAccessor accessor = createAccessor(opCtx);
return new LSMBTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
- componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+ componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
@Override
@@ -476,6 +475,6 @@
}
LSMBTreeRangeSearchCursor cursor = new LSMBTreeRangeSearchCursor(opCtx, returnDeletedTuples);
return new LSMBTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
- mergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+ mergeFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
index 7acc59f..2ad4fee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeFlushOperation.java
@@ -26,11 +26,13 @@
public class LSMBTreeFlushOperation extends FlushOperation {
private final FileReference bloomFilterFlushTarget;
+ private final LSMComponentFileReferences fileReferences;
public LSMBTreeFlushOperation(ILSMIndexAccessor accessor, FileReference flushTarget,
FileReference bloomFilterFlushTarget, ILSMIOOperationCallback callback, String indexIdentifier) {
super(accessor, flushTarget, callback, indexIdentifier);
this.bloomFilterFlushTarget = bloomFilterFlushTarget;
+ fileReferences = new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
}
public FileReference getBloomFilterTarget() {
@@ -39,6 +41,6 @@
@Override
public LSMComponentFileReferences getComponentFiles() {
- return new LSMComponentFileReferences(target, null, bloomFilterFlushTarget);
+ return fileReferences;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
index c7d555d..831562c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeMemoryComponent.java
@@ -28,9 +28,9 @@
private final BTree btree;
- public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc, boolean isActive,
+ public LSMBTreeMemoryComponent(LSMBTree lsmIndex, BTree btree, IVirtualBufferCache vbc,
ILSMComponentFilter filter) {
- super(lsmIndex, vbc, isActive, filter);
+ super(lsmIndex, vbc, filter);
this.btree = btree;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
index 1cfc414..1312e30 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeOpContext.java
@@ -32,7 +32,6 @@
import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.am.common.impls.IndexAccessParameters;
-import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
@@ -40,7 +39,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.MultiComparator;
import org.apache.hyracks.util.trace.ITracer;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 361612e..fb0b9ac0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -105,49 +105,55 @@
while (!outputPriorityQueue.isEmpty() || needPushElementIntoQueue) {
if (!outputPriorityQueue.isEmpty()) {
PriorityQueueElement queueHead = outputPriorityQueue.peek();
- if (canCallProceed && includeMutableComponent) {
- resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
- if (!resultOfSearchCallbackProceed) {
- // In case proceed() fails and there is an in-memory component,
- // we can't simply use this element since there might be a change.
- PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
- if (mutableElement != null) {
- // Copies the current queue head
- if (tupleBuilder == null) {
- tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+ if (canCallProceed) {
+ if (includeMutableComponent) {
+ resultOfSearchCallbackProceed = searchCallback.proceed(queueHead.getTuple());
+ if (!resultOfSearchCallbackProceed) {
+ // In case proceed() fails and there is an in-memory component,
+ // we can't simply use this element since there might be a change.
+ PriorityQueueElement mutableElement = remove(outputPriorityQueue, 0);
+ if (mutableElement != null) {
+ // Copies the current queue head
+ if (tupleBuilder == null) {
+ tupleBuilder = new ArrayTupleBuilder(cmp.getKeyFieldCount());
+ }
+ TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
+ copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+ // Unlatches/unpins the leaf page of the index.
+ rangeCursors[0].close();
+ // Reconcile.
+ searchCallback.reconcile(copyTuple);
+ // Re-traverses the index.
+ reusablePred.setLowKey(copyTuple, true);
+ btreeAccessors[0].search(rangeCursors[0], reusablePred);
+ pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
+ // now that we have completed the search and we have latches over the pages,
+ // it is safe to complete the operation.. but as per the API of the callback
+ // we only complete if we're producing this tuple
+ // get head again
+ queueHead = outputPriorityQueue.peek();
+ /*
+ * We need to restart in one of two cases:
+ * 1. no more elements in the priority queue.
+ * 2. the key of the head has changed (which means we need to call proceed)
+ */
+ if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
+ // cancel since we're not continuing
+ searchCallback.cancel(copyTuple);
+ continue;
+ }
+ searchCallback.complete(copyTuple);
+ // it is safe to proceed now
+ } else {
+ // There are no more elements in the memory component.. can safely skip locking for the
+ // remaining operations
+ includeMutableComponent = false;
}
- TupleUtils.copyTuple(tupleBuilder, queueHead.getTuple(), cmp.getKeyFieldCount());
- copyTuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
- // Unlatches/unpins the leaf page of the index.
- rangeCursors[0].close();
- // Reconcile.
- searchCallback.reconcile(copyTuple);
- // Re-traverses the index.
- reusablePred.setLowKey(copyTuple, true);
- btreeAccessors[0].search(rangeCursors[0], reusablePred);
- pushIntoQueueFromCursorAndReplaceThisElement(mutableElement);
- // now that we have completed the search and we have latches over the pages,
- // it is safe to complete the operation.. but as per the API of the callback
- // we only complete if we're producing this tuple
- // get head again
- queueHead = outputPriorityQueue.peek();
- /*
- * We need to restart in one of two cases:
- * 1. no more elements in the priority queue.
- * 2. the key of the head has changed (which means we need to call proceed)
- */
- if (queueHead == null || cmp.compare(copyTuple, queueHead.getTuple()) != 0) {
- // cancel since we're not continuing
- searchCallback.cancel(copyTuple);
- continue;
- }
- searchCallback.complete(copyTuple);
- // it is safe to proceed now
- } else {
- // There are no more elements in the memory component.. can safely skip locking for the
- // remaining operations
- includeMutableComponent = false;
}
+ } else {
+ // only perform locking for tuples from memory components.
+ // all tuples from disk components have already been committed, and we're safe to proceed
+ resultOfSearchCallbackProceed = true;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
index baf0d4a7..efacad1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java
@@ -107,6 +107,6 @@
@Override
public boolean getSearchOperationCallbackProceedResult() {
- return false;
+ return currentCursor.getSearchOperationCallbackProceedResult();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
index a1a1e45..ed2ee70 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeWithBuddyMemoryComponent.java
@@ -20,10 +20,10 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
/*
* This class is also not needed at the moment but is implemented anyway
@@ -34,8 +34,8 @@
private final BTree buddyBtree;
public LSMBTreeWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, BTree btree, BTree buddyBtree,
- IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
- super(lsmIndex, vbc, isActive, filter);
+ IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+ super(lsmIndex, vbc, filter);
this.btree = btree;
this.buddyBtree = buddyBtree;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
index cc10a98..d071bac 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/utils/LSMBTreeUtil.java
@@ -153,29 +153,24 @@
// can be used for both inserts and deletes
ITreeIndexFrameFactory transactionLeafFrameFactory =
new BTreeNSMLeafFrameFactory(transactionTupleWriterFactory);
-
TreeIndexFactory<DiskBTree> diskBTreeFactory =
new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
copyTupleLeafFrameFactory, cmpFactories, typeTraits.length);
TreeIndexFactory<DiskBTree> bulkLoadBTreeFactory = new DiskBTreeFactory(ioManager, diskBufferCache,
freePageManagerFactory, interiorFrameFactory, insertLeafFrameFactory, cmpFactories, typeTraits.length);
-
BloomFilterFactory bloomFilterFactory = new BloomFilterFactory(diskBufferCache, bloomFilterKeyFields);
-
// This is the component factory for transactions
TreeIndexFactory<DiskBTree> transactionBTreeFactory =
new DiskBTreeFactory(ioManager, diskBufferCache, freePageManagerFactory, interiorFrameFactory,
transactionLeafFrameFactory, cmpFactories, typeTraits.length);
//TODO remove BloomFilter from external dataset's secondary LSMBTree index
ILSMIndexFileManager fileNameManager = new LSMBTreeFileManager(ioManager, file, diskBTreeFactory, true);
-
ILSMDiskComponentFactory componentFactory =
new LSMBTreeWithBloomFilterDiskComponentFactory(diskBTreeFactory, bloomFilterFactory, null);
ILSMDiskComponentFactory bulkLoadComponentFactory =
new LSMBTreeWithBloomFilterDiskComponentFactory(bulkLoadBTreeFactory, bloomFilterFactory, null);
ILSMDiskComponentFactory transactionComponentFactory =
new LSMBTreeWithBloomFilterDiskComponentFactory(transactionBTreeFactory, bloomFilterFactory, null);
-
// the disk only index uses an empty ArrayList for virtual buffer caches
return new ExternalBTree(ioManager, interiorFrameFactory, insertLeafFrameFactory, deleteLeafFrameFactory,
diskBufferCache, fileNameManager, componentFactory, bulkLoadComponentFactory,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
index 107190d..329a54b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBloomFilterDiskComponent.java
@@ -59,13 +59,6 @@
}
@Override
- public void deactivateAndDestroy() throws HyracksDataException {
- super.deactivateAndDestroy();
- getBloomFilter().deactivate();
- getBloomFilter().destroy();
- }
-
- @Override
public void destroy() throws HyracksDataException {
super.destroy();
getBloomFilter().destroy();
@@ -78,9 +71,8 @@
}
@Override
- public void deactivateAndPurge() throws HyracksDataException {
- super.deactivateAndPurge();
- getBloomFilter().deactivate();
+ protected void purge() throws HyracksDataException {
+ super.purge();
getBloomFilter().purge();
}
@@ -92,10 +84,10 @@
}
@Override
- public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
- throws HyracksDataException {
- ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(fillFactor, verifyInput,
+ public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+ boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+ boolean cleanupEmptyComponent) throws HyracksDataException {
+ ChainedLSMDiskComponentBulkLoader chainedBulkLoader = super.createBulkLoader(operation, fillFactor, verifyInput,
numElementsHint, checkIfEmptyIndex, withFilter, cleanupEmptyComponent);
if (numElementsHint > 0) {
chainedBulkLoader.addBulkLoader(createBloomFilterBulkLoader(numElementsHint));
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
index c2f52e0..cace9e5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyDiskComponent.java
@@ -52,13 +52,6 @@
}
@Override
- public void deactivateAndDestroy() throws HyracksDataException {
- super.deactivateAndDestroy();
- getBuddyIndex().deactivate();
- getBuddyIndex().destroy();
- }
-
- @Override
public void destroy() throws HyracksDataException {
super.destroy();
getBuddyIndex().destroy();
@@ -71,9 +64,8 @@
}
@Override
- public void deactivateAndPurge() throws HyracksDataException {
- super.deactivateAndPurge();
- getBuddyIndex().deactivate();
+ protected void purge() throws HyracksDataException {
+ super.purge();
getBuddyIndex().purge();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
deleted file mode 100644
index 551303f..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/AbstractLSMWithBuddyMemoryComponent.java
+++ /dev/null
@@ -1,63 +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.hyracks.storage.am.lsm.common.api;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMMemoryComponent;
-
-public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
-
- public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
- ILSMComponentFilter filter) {
- super(lsmIndex, vbc, isActive, filter);
- }
-
- public abstract AbstractTreeIndex getBuddyIndex();
-
- @Override
- public void doReset() throws HyracksDataException {
- super.doReset();
- getBuddyIndex().deactivate();
- getBuddyIndex().destroy();
- getBuddyIndex().create();
- getBuddyIndex().activate();
- }
-
- @Override
- public void doAllocate() throws HyracksDataException {
- super.doAllocate();
- getBuddyIndex().create();
- getBuddyIndex().activate();
- }
-
- @Override
- public void doDeallocate() throws HyracksDataException {
- super.doDeallocate();
- getBuddyIndex().deactivate();
- getBuddyIndex().destroy();
- }
-
- @Override
- public void validate() throws HyracksDataException {
- super.validate();
- getBuddyIndex().validate();
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
new file mode 100644
index 0000000..a9b9cc0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IIoOperationFailedCallback.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+/**
+ * A callback that will be called on failures of IO operations
+ */
+public interface IIoOperationFailedCallback {
+ /**
+ * Called on a failed IO operation
+ *
+ * @param operation
+ * the failed operation
+ * @param failure
+ * the failure
+ */
+ void operationFailed(ILSMIOOperation operation, Throwable failure);
+
+ /**
+ * Called on a scheduler failure.
+ *
+ * @param scheduler
+ * the failed scheduler
+ * @param failure
+ * the failure
+ */
+ void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
index fc55ce5..fef59e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponent.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.common.api;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.common.IIndex;
/**
@@ -139,4 +140,17 @@
* @throws HyracksDataException
*/
ILSMComponentId getId() throws HyracksDataException;
+
+ /**
+ * Prepare the component to be scheduled for an IO operation
+ *
+ * @param ioOperationType
+ * @throws HyracksDataException
+ */
+ void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException;
+
+ /**
+ * @return the number of readers inside a component
+ */
+ int getReaderCount();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
index 5662862..c3835eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentId.java
@@ -38,4 +38,14 @@
boolean missing();
IdCompareResult compareTo(ILSMComponentId id);
+
+ /**
+ * @return the min Id
+ */
+ long getMinId();
+
+ /**
+ * @return the max Id
+ */
+ long getMaxId();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
index 5dd3061..e6aa2d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGenerator.java
@@ -26,12 +26,17 @@
/**
* @return An Id for LSM component
*/
- public ILSMComponentId getId();
+ ILSMComponentId getId();
/**
* Refresh the component Id generator to generate the next Id.
* {@link #getId()} would always return the same Id before this method is called.
*/
- public void refresh();
+ void refresh();
+
+ /**
+ * @return the index of the current memory component
+ */
+ int getCurrentComponentIndex();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
deleted file mode 100644
index 4ec82c1..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,31 +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.hyracks.storage.am.lsm.common.api;
-
-import java.io.Serializable;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.common.IResource;
-
-@FunctionalInterface
-public interface ILSMComponentIdGeneratorFactory extends Serializable {
- ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource)
- throws HyracksDataException;
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
index bd2bb45..1500f37 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponent.java
@@ -50,6 +50,7 @@
/**
* @return LsmIndex of the component
*/
+ @Override
AbstractLSMIndex getLsmIndex();
/**
@@ -142,6 +143,7 @@
* Creates a bulkloader pipeline which includes all chained operations, bulkloading individual elements of the
* component: indexes, LSM filters, Bloom filters, buddy indexes, etc.
*
+ * @param operation
* @param fillFactor
* @param verifyInput
* @param numElementsHint
@@ -151,6 +153,7 @@
* @return
* @throws HyracksDataException
*/
- ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent) throws HyracksDataException;
+ ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
+ throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
index 13ce971..8284cee 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMDiskComponentBulkLoader.java
@@ -41,4 +41,9 @@
*/
void cleanupArtifacts() throws HyracksDataException;
+ /**
+ * @return The operation this bulkLoader is operating under
+ */
+ ILSMIOOperation getOperation();
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
index b32dd0f..c4a0352 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMHarness.java
@@ -108,7 +108,7 @@
* @throws HyracksDataException
* @throws IndexException
*/
- void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+ ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
/**
* Schedule full merge
@@ -118,17 +118,16 @@
* @throws HyracksDataException
* @throws IndexException
*/
- void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+ ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException;
/**
* Perform a merge operation
*
- * @param ctx
* @param operation
* @throws HyracksDataException
* @throws IndexException
*/
- void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+ void merge(ILSMIOOperation operation) throws HyracksDataException;
/**
* Schedule a flush
@@ -137,17 +136,16 @@
* @param callback
* @throws HyracksDataException
*/
- void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+ ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException;
/**
* Perform a flush
*
- * @param ctx
* @param operation
* @throws HyracksDataException
* @throws IndexException
*/
- void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException;
+ void flush(ILSMIOOperation operation) throws HyracksDataException;
/**
* Add bulk loaded component
@@ -171,13 +169,11 @@
* the operation context
* @param diskComponents
* the disk component to be replicated
- * @param bulkload
- * true if the components were bulk loaded, false otherwise
* @param opType
* The operation type
* @throws HyracksDataException
*/
- void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+ void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
LSMOperationType opType) throws HyracksDataException;
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 65e7f64..3245455 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -18,14 +18,16 @@
*/
package org.apache.hyracks.storage.am.lsm.common.api;
+import java.util.Map;
import java.util.concurrent.Callable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
-public interface ILSMIOOperation extends Callable<Boolean> {
+public interface ILSMIOOperation extends Callable<LSMIOOperationStatus> {
/**
* Represents the io operation type
@@ -38,6 +40,20 @@
}
/**
+ * Represents the status of the IO operation
+ */
+ enum LSMIOOperationStatus {
+ /**
+ * Operation successful
+ */
+ SUCCESS,
+ /**
+ * Operation failed
+ */
+ FAILURE
+ }
+
+ /**
* @return the device on which the operation is running
*/
IODeviceHandle getDevice();
@@ -58,7 +74,7 @@
LSMIOOperationType getIOOpertionType();
@Override
- Boolean call() throws HyracksDataException;
+ LSMIOOperationStatus call() throws HyracksDataException;
/**
* @return The target of the io operation
@@ -74,4 +90,62 @@
* @return the component files produced by this operation
*/
LSMComponentFileReferences getComponentFiles();
+
+ /**
+ * @return the failure in the io operation if any, null otherwise
+ */
+ Throwable getFailure();
+
+ /**
+ * @return set the failure in the io operation
+ */
+ void setFailure(Throwable failure);
+
+ /**
+ * @return the status of the IO operation
+ */
+ LSMIOOperationStatus getStatus();
+
+ /**
+ * Set the status of the IO operation
+ *
+ * @param status
+ */
+ void setStatus(LSMIOOperationStatus status);
+
+ /**
+ * @return the new component produced by this operation if any, null otherwise
+ */
+ ILSMDiskComponent getNewComponent();
+
+ /**
+ * Set the new component produced by this operation
+ *
+ * @param component
+ */
+ void setNewComponent(ILSMDiskComponent component);
+
+ /**
+ * Destroy the operation after the scheduler is done with it
+ */
+ void complete();
+
+ /**
+ * Wait for the operation to complete
+ *
+ * @throws InterruptedException
+ */
+ void sync() throws InterruptedException;
+
+ /**
+ * Add a listener for operation complete event
+ *
+ * @param listener
+ */
+ void addCompleteListener(IoOperationCompleteListener listener);
+
+ /**
+ * Get parameters passed when calling this IO operation
+ */
+ Map<String, Object> getParameters();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
index acc9e89..e448ae0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallback.java
@@ -23,14 +23,23 @@
public interface ILSMIOOperationCallback {
/**
- * This method is called on an IO operation before the operation starts.
- * (i.e. IO operations could be flush, or merge operations.)
- * For flush, this is called immediately before switching the current memory component pointer
+ * This method is called on an IO operation before the operation is scheduled
+ * For operations that are not scheduled(i,e. Bulk load), this call is skipped.
+ *
+ * @param operation
+ * @throws HyracksDataException
*/
- void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+ void scheduled(ILSMIOOperation operation) throws HyracksDataException;
/**
- * This method is called on an IO operation sometime after the operation was completed.
+ * This method is called on an IO operation before the operation starts.
+ * (i.e. IO operations could be flush, or merge operations.)
+ */
+ void beforeOperation(ILSMIOOperation operation) throws HyracksDataException;
+
+ /**
+ * This method is called on an IO operation sometime after the operation is completed but before the new component
+ * is marked as valid.
* (i.e. IO operations could be flush or merge operations.)
*
* Copying content of metadata page from memory component to disk component should be done in this call
@@ -38,31 +47,37 @@
*
* @throws HyracksDataException
*/
- void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+ void afterOperation(ILSMIOOperation operation) throws HyracksDataException;
/**
* This method is called on an IO operation when the operation needs any cleanup works
* regardless that the IO operation was executed or not. Once the IO operation is executed,
* this method should be called after ILSMIOOperationCallback.afterOperation() was called.
*
+ */
+ void afterFinalize(ILSMIOOperation operation) throws HyracksDataException;
+
+ /**
+ * This method is called after the schduler is done with the IO operation
+ * For operation that are not scheduled, this call is skipped
+ *
+ * @param operation
* @throws HyracksDataException
*/
- void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException;
+ void completed(ILSMIOOperation operation);
/**
* This method is called when a memory component is recycled
*
* @param component
- * @param componentSwitched
- * true if the component index was advanced for this recycle, false otherwise
*/
- void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException;
+ void recycled(ILSMMemoryComponent component) throws HyracksDataException;
/**
* This method is called when a memory component is allocated
*
* @param component
+ * the allocated component
*/
void allocated(ILSMMemoryComponent component) throws HyracksDataException;
-
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
index e8742b5..8ccf41f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationCallbackFactory.java
@@ -32,5 +32,18 @@
*/
void initialize(INCServiceContext ncCtx, IResource resource);
+ /**
+ * Create the IO Operation Callback
+ *
+ * @param index
+ * @return
+ * @throws HyracksDataException
+ */
ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException;
+
+ /**
+ * @return the current memory component index
+ * @throws HyracksDataException
+ */
+ int getCurrentMemoryComponentIndex() throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
index a8467d3..5cb05a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndex.java
@@ -20,6 +20,7 @@
package org.apache.hyracks.storage.am.lsm.common.api;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.replication.IReplicationJob.ReplicationOperation;
@@ -28,6 +29,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.LSMHarness;
import org.apache.hyracks.storage.common.IIndex;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
@@ -48,8 +50,6 @@
ILSMOperationTracker getOperationTracker();
- ILSMIOOperationScheduler getIOScheduler();
-
ILSMIOOperationCallback getIOOperationCallback();
/**
@@ -74,11 +74,33 @@
public void scanDiskComponents(ILSMIndexOperationContext ctx, IIndexCursor cursor) throws HyracksDataException;
- void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+ /**
+ * Create a flush operation.
+ * This is an atomic operation. If an exception is thrown, no partial effect is left
+ *
+ * @return the flush operation
+ *
+ * @param ctx
+ * the operation context
+ * @param callback
+ * the IO callback
+ * @throws HyracksDataException
+ */
+ ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException;
- void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback) throws HyracksDataException;
+ /**
+ * Create a merge operation.
+ * This is an atomic operation. If an exception is thrown, no partial effect is left
+ *
+ * @param ctx
+ * the operation context
+ * @param callback
+ * the IO callback
+ * @throws HyracksDataException
+ */
+ ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException;
ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException;
@@ -108,7 +130,7 @@
boolean isCurrentMutableComponentEmpty() throws HyracksDataException;
- void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents, boolean bulkload,
+ void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> diskComponents,
ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException;
boolean isMemoryComponentsAllocated();
@@ -157,4 +179,36 @@
* @return the {@link ILSMHarness} of the index
*/
ILSMHarness getHarness();
+
+ /**
+ * Cleanup the files of the failed operation
+ *
+ * @param operation
+ */
+ void cleanUpFilesForFailedOperation(ILSMIOOperation operation);
+
+ /**
+ * @return the absolute path of the index
+ */
+ String getIndexIdentifier();
+
+ /**
+ * Create a bulk loader
+ *
+ * @param fillFactor
+ * @param verifyInput
+ * @param numElementsHint
+ * @param checkIfEmptyIndex
+ * @param parameters
+ * @return
+ * @throws HyracksDataException
+ */
+ IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException;
+
+ /**
+ * Reset the current memory component id to 0.
+ */
+ void resetCurrentComponentIndex();
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
index 61ef6cf..42d3ab7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexAccessor.java
@@ -44,34 +44,27 @@
/**
* Schedule a flush operation
*
- * @param callback
- * the IO operation callback
* @throws HyracksDataException
*/
- void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException;
+ ILSMIOOperation scheduleFlush() throws HyracksDataException;
/**
* Schedule a merge operation
*
- * @param callback
- * the merge operation callback
* @param components
* the components to be merged
* @throws HyracksDataException
* @throws IndexException
*/
- void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
- throws HyracksDataException;
+ ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException;
/**
* Schedule a full merge
*
- * @param callback
- * the merge operation callback
* @throws HyracksDataException
* @throws IndexException
*/
- void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException;
+ ILSMIOOperation scheduleFullMerge() throws HyracksDataException;
/**
* Delete the tuple from the memory component only. Don't replace with antimatter tuple
@@ -193,17 +186,15 @@
*
* @param diskComponents
* the components to be replicated
- * @param bulkload
- * true if the components were bulkloaded, false otherwise
* @param opType
* the operation type
* @throws HyracksDataException
*/
- void scheduleReplication(List<ILSMDiskComponent> diskComponents, boolean bulkload, LSMOperationType opType)
+ void scheduleReplication(List<ILSMDiskComponent> diskComponents, LSMOperationType opType)
throws HyracksDataException;
/**
- * Force a flush of the in-memory component.
+ * Flush an in-memory component.
*
* @throws HyracksDataException
* @throws TreeIndexException
@@ -265,9 +256,10 @@
void deleteComponents(Predicate<ILSMComponent> predicate) throws HyracksDataException;
/**
- * Update the filter of an LSM index
- * @param tuple
- * @throws HyracksDataException
- */
+ * Update the filter of an LSM index
+ *
+ * @param tuple
+ * @throws HyracksDataException
+ */
void updateFilter(ITupleReference tuple) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
index b34b403..238d5f8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexOperationContext.java
@@ -19,12 +19,11 @@
package org.apache.hyracks.storage.am.lsm.common.api;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -94,26 +93,26 @@
void setRecovery(boolean recovery);
/**
- * @return the IO Operation type associated with this context
+ * @return the IO operation associated with this context
*/
- LSMIOOperationType getIoOperationType();
+ ILSMIOOperation getIoOperation();
/**
- * Set the IO Operation type associated with this context
+ * Set the IO operation associated with this context
*
- * @param ioOpType
+ * @param ioOperation
*/
- void setIoOperationType(LSMIOOperationType ioOpType);
+ void setIoOperation(ILSMIOOperation ioOperation);
/**
- * @return the new component produced by this operation if any, null otherwise
- */
- ILSMDiskComponent getNewComponent();
-
- /**
- * Set the new component produced by this operation
+ * Set a map in the context to pass pairs of keys and values
*
- * @param component
+ * @param map
*/
- void setNewComponent(ILSMDiskComponent component);
+ void setParameters(Map<String, Object> map);
+
+ /**
+ * @return the key value map of the context
+ */
+ Map<String, Object> getParameters();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
index 4ff6377..c86f7b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMMemoryComponent.java
@@ -63,19 +63,6 @@
void setModified();
/**
- * request the component to be active
- */
- void requestActivation();
-
- /**
- * Set the component state
- *
- * @param state
- * the new state
- */
- void setState(ComponentState state);
-
- /**
* Allocates memory to this component, create and activate it.
* This method is atomic. If an exception is thrown, then the call had no effect.
*
@@ -108,8 +95,14 @@
*
* @param newId
* @param force
- * Whether to force reset the Id to skip sanity checks
+ * Whether to force reset the Id to skip sanity checks
* @throws HyracksDataException
*/
void resetId(ILSMComponentId newId, boolean force) throws HyracksDataException;
+
+ /**
+ * Set the component state to be unwritable to prevent future writers from non-force
+ * entry to the component
+ */
+ void setUnwritable();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
index fc1d5f4..74a5d60 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ITwoPCIndex.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.common.api;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -30,8 +31,8 @@
* This function is used to create a BulkLoader for a transaction that is capable of insertions and deletions
* and the bulk loaded component is hidden from the index
*/
- public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException;
+ public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException;
/**
* This function is used to commit the previous transaction if it was resulted in creating any components
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
new file mode 100644
index 0000000..e5ba81e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/IoOperationCompleteListener.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.common.api;
+
+public interface IoOperationCompleteListener {
+
+ /**
+ * Called when an IO operation completes
+ *
+ * @param operation
+ */
+ void completed(ILSMIOOperation operation);
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
index 63d2697..e200bfd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/LSMOperationType.java
@@ -22,6 +22,7 @@
SEARCH,
MODIFICATION,
FORCE_MODIFICATION,
+ LOAD,
FLUSH,
MERGE,
REPLICATE,
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
index 4e8cf71..682ffef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexCompactOperatorNodePushable.java
@@ -57,7 +57,7 @@
indexHelper.open();
ILSMIndex index = (ILSMIndex) indexHelper.getIndexInstance();
ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFullMerge(index.getIOOperationCallback());
+ accessor.scheduleFullMerge();
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
index aee46f0..fc9a362 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
@@ -18,11 +18,18 @@
*/
package org.apache.hyracks.storage.am.lsm.common.impls;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.api.util.ExceptionUtils;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
public abstract class AbstractIoOperation implements ILSMIOOperation {
@@ -30,6 +37,11 @@
protected final FileReference target;
protected final ILSMIOOperationCallback callback;
protected final String indexIdentifier;
+ private Throwable failure;
+ private LSMIOOperationStatus status = LSMIOOperationStatus.SUCCESS;
+ private ILSMDiskComponent newComponent;
+ private boolean completed = false;
+ private List<IoOperationCompleteListener> completeListeners;
public AbstractIoOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
String indexIdentifier) {
@@ -63,4 +75,75 @@
public String getIndexIdentifier() {
return indexIdentifier;
}
+
+ @Override
+ public Throwable getFailure() {
+ return failure;
+ }
+
+ @Override
+ public void setFailure(Throwable failure) {
+ status = LSMIOOperationStatus.FAILURE;
+ this.failure = ExceptionUtils.suppress(this.failure, failure);
+ }
+
+ @Override
+ public LSMIOOperationStatus getStatus() {
+ return status;
+ }
+
+ @Override
+ public void setStatus(LSMIOOperationStatus status) {
+ this.status = status;
+ }
+
+ @Override
+ public ILSMDiskComponent getNewComponent() {
+ return newComponent;
+ }
+
+ @Override
+ public void setNewComponent(ILSMDiskComponent component) {
+ this.newComponent = component;
+ }
+
+ @Override
+ public synchronized void complete() {
+ if (completed) {
+ throw new IllegalStateException("Multiple destroy calls");
+ }
+ callback.completed(this);
+ completed = true;
+ if (completeListeners != null) {
+ for (IoOperationCompleteListener listener : completeListeners) {
+ listener.completed(this);
+ }
+ completeListeners = null;
+ }
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void sync() throws InterruptedException {
+ while (!completed) {
+ wait();
+ }
+ }
+
+ @Override
+ public Map<String, Object> getParameters() {
+ return accessor.getOpContext().getParameters();
+ }
+
+ @Override
+ public synchronized void addCompleteListener(IoOperationCompleteListener listener) {
+ if (completed) {
+ listener.completed(this);
+ } else {
+ if (completeListeners == null) {
+ completeListeners = new LinkedList<>();
+ }
+ completeListeners.add(listener);
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
index 84d2fe5..574a371 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMComponent.java
@@ -51,4 +51,9 @@
public final AbstractLSMIndex getLsmIndex() {
return lsmIndex;
}
+
+ @Override
+ public int getReaderCount() {
+ return readerCount;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
index c4616d3..aa312fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMDiskComponent.java
@@ -24,6 +24,8 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
import org.apache.hyracks.storage.am.lsm.common.util.LSMComponentIdUtils;
@@ -52,27 +54,34 @@
}
@Override
+ public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+ if (ioOperationType != LSMIOOperationType.MERGE) {
+ throw new IllegalStateException("Unsupported operation type: " + ioOperationType);
+ }
+ if (state == ComponentState.INACTIVE) {
+ throw new IllegalStateException("Trying to schedule a merge of an inactive disk component");
+ }
+ if (state == ComponentState.READABLE_MERGING) {
+ // This should never happen unless there are two concurrent merges that were scheduled
+ // concurrently and they have interleaving components to be merged.
+ // This should be handled properly by the merge policy, but we guard against that here anyway.
+ throw new IllegalStateException("The disk component has already been scheduled for a merge");
+ }
+ state = ComponentState.READABLE_MERGING;
+ }
+
+ @Override
public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) {
if (state == ComponentState.INACTIVE) {
throw new IllegalStateException("Trying to enter an inactive disk component");
}
-
switch (opType) {
case FORCE_MODIFICATION:
case MODIFICATION:
case REPLICATE:
case SEARCH:
case DISK_COMPONENT_SCAN:
- readerCount++;
- break;
case MERGE:
- if (state == ComponentState.READABLE_MERGING) {
- // This should never happen unless there are two concurrent merges that were scheduled
- // concurrently and they have interleaving components to be merged.
- // This should be handled properly by the merge policy, but we guard against that here anyway.
- return false;
- }
- state = ComponentState.READABLE_MERGING;
readerCount++;
break;
default:
@@ -86,19 +95,22 @@
throws HyracksDataException {
switch (opType) {
case MERGE:
+ readerCount--;
// In case two merge operations were scheduled to merge an overlapping set of components,
// the second merge will fail and it must reset those components back to their previous state.
if (failedOperation) {
state = ComponentState.READABLE_UNWRITABLE;
+ } else {
+ state = (readerCount == 0) ? ComponentState.INACTIVE : ComponentState.UNREADABLE_UNWRITABLE;
}
- // Fallthrough
+ break;
case FORCE_MODIFICATION:
case MODIFICATION:
case REPLICATE:
case SEARCH:
case DISK_COMPONENT_SCAN:
readerCount--;
- if (readerCount == 0 && state == ComponentState.READABLE_MERGING) {
+ if (readerCount == 0 && state == ComponentState.UNREADABLE_UNWRITABLE) {
state = ComponentState.INACTIVE;
}
break;
@@ -163,9 +175,15 @@
}
@Override
- public void deactivateAndDestroy() throws HyracksDataException {
- getIndex().deactivate();
- getIndex().destroy();
+ public final void deactivateAndDestroy() throws HyracksDataException {
+ deactivateAndPurge();
+ destroy();
+ }
+
+ @Override
+ public final void deactivateAndPurge() throws HyracksDataException {
+ deactivate();
+ purge();
}
@Override
@@ -178,9 +196,7 @@
getIndex().deactivate();
}
- @Override
- public void deactivateAndPurge() throws HyracksDataException {
- getIndex().deactivate();
+ protected void purge() throws HyracksDataException {
getIndex().purge();
}
@@ -203,17 +219,27 @@
getIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
}
+ /**
+ * Allows sub-class extend this method to use specialized bulkloader for merge
+ */
+ protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+ return this.createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ }
+
@Override
- public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
- throws HyracksDataException {
+ public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+ boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+ boolean cleanupEmptyComponent) throws HyracksDataException {
ChainedLSMDiskComponentBulkLoader chainedBulkLoader =
- new ChainedLSMDiskComponentBulkLoader(this, cleanupEmptyComponent);
+ new ChainedLSMDiskComponentBulkLoader(operation, this, cleanupEmptyComponent);
if (withFilter && getLsmIndex().getFilterFields() != null) {
chainedBulkLoader.addBulkLoader(createFilterBulkLoader());
}
- chainedBulkLoader
- .addBulkLoader(createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex));
+ IChainedComponentBulkLoader indexBulkloader = operation.getIOOpertionType() == LSMIOOperationType.MERGE
+ ? createMergeIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex)
+ : createIndexBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ chainedBulkLoader.addBulkLoader(indexBulkloader);
return chainedBulkLoader;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
index 2db2f79..c9fb328 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java
@@ -21,9 +21,11 @@
import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@@ -41,6 +43,7 @@
import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
import org.apache.hyracks.storage.am.lsm.common.api.IComponentFilterHelper;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilterFrameFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
@@ -74,7 +77,6 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final ILSMHarness lsmHarness;
protected final IIOManager ioManager;
- protected final ILSMIOOperationScheduler ioScheduler;
protected final ILSMIOOperationCallback ioOpCallback;
// In-memory components.
@@ -95,6 +97,7 @@
protected final int[] filterFields;
protected final boolean durable;
protected boolean isActive;
+ protected volatile boolean isDeactivating = false;
protected final AtomicBoolean[] flushRequests;
protected volatile boolean memoryComponentsAllocated = false;
protected ITracer tracer;
@@ -102,6 +105,7 @@
protected final ILSMDiskComponentFactory componentFactory;
// Factory for creating on-disk index components during bulkload.
protected final ILSMDiskComponentFactory bulkLoadComponentFactory;
+ private int numScheduledFlushes = 0;
public AbstractLSMIndex(IIOManager ioManager, List<IVirtualBufferCache> virtualBufferCaches,
IBufferCache diskBufferCache, ILSMIndexFileManager fileManager, double bloomFilterFalsePositiveRate,
@@ -115,7 +119,6 @@
this.diskBufferCache = diskBufferCache;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
- this.ioScheduler = ioScheduler;
this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
this.componentFactory = componentFactory;
this.bulkLoadComponentFactory = bulkLoadComponentFactory;
@@ -127,11 +130,12 @@
this.inactiveDiskComponents = new LinkedList<>();
this.durable = durable;
this.tracer = tracer;
- lsmHarness = new LSMHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(), tracer);
+ lsmHarness = new LSMHarness(this, ioScheduler, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled(),
+ tracer);
isActive = false;
diskComponents = new ArrayList<>();
memoryComponents = new ArrayList<>();
- currentMutableComponentId = new AtomicInteger();
+ currentMutableComponentId = new AtomicInteger(ioOpCallbackFactory.getCurrentMemoryComponentIndex());
flushRequests = new AtomicBoolean[virtualBufferCaches.size()];
for (int i = 0; i < virtualBufferCaches.size(); i++) {
flushRequests[i] = new AtomicBoolean();
@@ -148,13 +152,13 @@
this.diskBufferCache = diskBufferCache;
this.fileManager = fileManager;
this.bloomFilterFalsePositiveRate = bloomFilterFalsePositiveRate;
- this.ioScheduler = ioScheduler;
this.ioOpCallback = ioOpCallbackFactory.createIoOpCallback(this);
this.componentFactory = componentFactory;
this.bulkLoadComponentFactory = bulkLoadComponentFactory;
this.durable = durable;
this.tracer = tracer;
- lsmHarness = new ExternalIndexHarness(this, mergePolicy, opTracker, diskBufferCache.isReplicationEnabled());
+ lsmHarness = new ExternalIndexHarness(this, ioScheduler, mergePolicy, opTracker,
+ diskBufferCache.isReplicationEnabled());
isActive = false;
diskComponents = new LinkedList<>();
this.inactiveDiskComponents = new LinkedList<>();
@@ -205,28 +209,35 @@
deactivate(true);
}
+ @SuppressWarnings({ "squid:S1181", "squid:S2142" })
@Override
public synchronized void deactivate(boolean flush) throws HyracksDataException {
if (!isActive) {
throw HyracksDataException.create(ErrorCode.CANNOT_DEACTIVATE_INACTIVE_INDEX);
}
- if (flush) {
- flushMemoryComponent();
- }
- deactivateDiskComponents();
- deallocateMemoryComponents();
- isActive = false;
- }
-
- private void flushMemoryComponent() throws HyracksDataException {
- BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(ioOpCallback);
- ILSMIndexAccessor accessor = createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFlush(cb);
+ // The following member is used to prevent scheduling of new merges as memory components
+ // get flushed. This now works only if the caller of deactivate waited for all IO
+ // operations to complete. Otherwise, disk components can be evicted while background
+ // merges are ongoing.
+ isDeactivating = true;
try {
- cb.waitForIO();
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- throw HyracksDataException.create(e);
+ LOGGER.log(Level.INFO, "Deactivating the index: {}. STARTED", this);
+ if (flush && memoryComponentsAllocated) {
+ try {
+ createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFlush().sync();
+ } catch (InterruptedException e) {
+ throw HyracksDataException.create(e);
+ }
+ LOGGER.log(Level.INFO, "Deactivating the index: {}. Flushed", this);
+ }
+ LOGGER.log(Level.INFO, "Deactivating the disk components of: {}", this);
+ deactivateDiskComponents();
+ LOGGER.log(Level.INFO, "Deallocating memory components of: {}", this);
+ deallocateMemoryComponents();
+ isActive = false;
+ LOGGER.log(Level.INFO, "Deactivating the index: {}. COMPLETED", this);
+ } finally {
+ isDeactivating = false;
}
}
@@ -282,6 +293,8 @@
c.reset();
}
}
+ numScheduledFlushes = 0;
+ currentMutableComponentId.set(0);
}
@Override
@@ -297,19 +310,18 @@
switch (ctx.getOperation()) {
case UPDATE:
case PHYSICALDELETE:
- case FLUSH:
- case DELETE_MEMORY_COMPONENT:
+ case DELETE_COMPONENTS:
case DELETE:
case UPSERT:
operationalComponents.add(memoryComponents.get(cmc));
break;
case INSERT:
- addOperationalMutableComponents(operationalComponents, true);
+ addOperationalMemoryComponents(operationalComponents, true);
operationalComponents.addAll(diskComponents);
break;
case SEARCH:
if (memoryComponentsAllocated) {
- addOperationalMutableComponents(operationalComponents, false);
+ addOperationalMemoryComponents(operationalComponents, false);
}
if (filterManager != null) {
for (int i = 0; i < diskComponents.size(); i++) {
@@ -326,13 +338,6 @@
}
break;
- case MERGE:
- case DELETE_DISK_COMPONENTS:
- operationalComponents.addAll(ctx.getComponentsToBeMerged());
- break;
- case FULL_MERGE:
- operationalComponents.addAll(diskComponents);
- break;
case REPLICATE:
operationalComponents.addAll(ctx.getComponentsToBeReplicated());
break;
@@ -350,65 +355,169 @@
}
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
- AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
- opCtx.setOperation(ctx.getOperation());
- opCtx.getComponentHolder().addAll(ctx.getComponentHolder());
- ILSMIOOperation flushOp = createFlushOperation(opCtx, componentFileRefs, callback);
- ioScheduler.scheduleOperation(TracedIOOperation.wrap(flushOp, tracer));
+ public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ ILSMMemoryComponent flushingComponent = getCurrentMemoryComponent();
+ if (flushingComponent.getWriterCount() > 0) {
+ throw new IllegalStateException(
+ "createFlushOperation is called on a component with writers: " + flushingComponent);
+ }
+ // take care of the flush cycling
+ ILSMIOOperation flushOp =
+ TracedIOOperation.wrap(createFlushOperation(createOpContext(NoOpIndexAccessParameters.INSTANCE),
+ fileManager.getRelFlushFileReference(), ioOpCallback), tracer);
+ // Changing the flush status should *always* precede changing the mutable component.
+ flushingComponent.schedule(LSMIOOperationType.FLUSH);
+ numScheduledFlushes++;
+ changeFlushStatusForCurrentMutableCompoent(false);
+ changeMutableComponent();
+ ILSMIndexAccessor accessor = flushOp.getAccessor();
+ ILSMIndexOperationContext flushCtx = accessor.getOpContext();
+ flushCtx.setOperation(ctx.getOperation()); // Could be component delete
+ flushCtx.getComponentHolder().add(flushingComponent);
+ flushCtx.setIoOperation(flushOp);
+ propagateMap(ctx, flushCtx);
+ ioOpCallback.scheduled(flushOp);
+ return flushOp;
}
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
- // merge must create a different op ctx
- AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
- opCtx.setOperation(ctx.getOperation());
- opCtx.getComponentHolder().addAll(mergingComponents);
- mergingComponents.stream().map(ILSMDiskComponent.class::cast).forEach(opCtx.getComponentsToBeMerged()::add);
- ILSMDiskComponent firstComponent = (ILSMDiskComponent) mergingComponents.get(0);
- ILSMDiskComponent lastComponent = (ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1);
- LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
- ILSMIOOperation mergeOp = createMergeOperation(opCtx, mergeFileRefs, callback);
- ioScheduler.scheduleOperation(TracedIOOperation.wrap(mergeOp, tracer));
- }
-
- private void addOperationalMutableComponents(List<ILSMComponent> operationalComponents, boolean modification) {
- int cmc = currentMutableComponentId.get();
- int numMutableComponents = memoryComponents.size();
- for (int i = 0; i < numMutableComponents - 1; i++) {
- ILSMMemoryComponent c = memoryComponents.get((cmc + i + 1) % numMutableComponents);
- if (c.isReadable()) {
- // Make sure newest components are added first if readable
- operationalComponents.add(0, c);
+ public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ List<ILSMDiskComponent> mergingComponents = ctx.getComponentsToBeMerged();
+ // Merge operation can fail if another merge is already scheduled on those components
+ // This should be guarded against by the merge policy but we still protect against here
+ if (isDeactivating
+ || (mergingComponents.size() < 2 && ctx.getOperation() != IndexOperation.DELETE_COMPONENTS)) {
+ return NoOpIoOperation.INSTANCE;
+ }
+ for (int i = 0; i < mergingComponents.size(); i++) {
+ if (mergingComponents.get(i).getState() == ComponentState.READABLE_MERGING) {
+ return NoOpIoOperation.INSTANCE;
}
}
- // The current mutable component is added if modification operation or if readable
- // This ensures that activation of new component only happens in case of modifications
- // and allow for controlling that without stopping search operations
- ILSMMemoryComponent c = memoryComponents.get(cmc);
- if (modification || c.isReadable()) {
- operationalComponents.add(0, c);
+ // merge must create a different op ctx
+ AbstractLSMIndexOperationContext mergeCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
+ mergeCtx.setOperation(ctx.getOperation());
+ mergeCtx.getComponentHolder().addAll(mergingComponents);
+ propagateMap(ctx, mergeCtx);
+ mergingComponents.stream().forEach(mergeCtx.getComponentsToBeMerged()::add);
+ ILSMDiskComponent firstComponent = mergingComponents.get(0);
+ ILSMDiskComponent lastComponent = mergingComponents.get(mergingComponents.size() - 1);
+ LSMComponentFileReferences mergeFileRefs = getMergeFileReferences(firstComponent, lastComponent);
+ ILSMIOOperation mergeOp =
+ TracedIOOperation.wrap(createMergeOperation(mergeCtx, mergeFileRefs, ioOpCallback), tracer);
+ mergeCtx.setIoOperation(mergeOp);
+ for (int i = 0; i < mergingComponents.size(); i++) {
+ mergingComponents.get(i).schedule(LSMIOOperationType.MERGE);
}
+ ioOpCallback.scheduled(mergeOp);
+ return mergeOp;
+ }
+
+ private static void propagateMap(ILSMIndexOperationContext src, ILSMIndexOperationContext destination) {
+ Map<String, Object> map = src.getParameters();
+ if (map != null && !map.isEmpty()) {
+ destination.setParameters(new HashMap<>(map));
+ }
+ }
+
+ private void addOperationalMemoryComponents(List<ILSMComponent> operationalComponents, boolean modification) {
+ // add current memory component first if needed
+ if (numScheduledFlushes < memoryComponents.size()) {
+ ILSMMemoryComponent c = memoryComponents.get(currentMutableComponentId.get());
+ // The current mutable component is added if modification or readable
+ // This ensures that activation of new component only happens in case of modifications
+ // and allow for controlling that without stopping search operations
+ if (modification || c.isReadable()) {
+ operationalComponents.add(c);
+ }
+ }
+ if (modification && numScheduledFlushes >= memoryComponents.size()) {
+ // will fail the enterComponent call and retry
+ operationalComponents.add(memoryComponents.get(0));
+ return;
+ }
+ addImmutableMemoryComponents(operationalComponents);
+ }
+
+ private void addImmutableMemoryComponents(List<ILSMComponent> operationalComponents) {
+ int cmc = currentMutableComponentId.get();
+ int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+ int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+ for (int i = 0; i < numImmutableMemoryComponents; i++) {
+ next--;
+ if (next < 0) {
+ next = memoryComponents.size() - 1;
+ }
+ //newer components first
+ ILSMMemoryComponent c = memoryComponents.get(next);
+ if (c.isReadable()) {
+ operationalComponents.add(c);
+ }
+ }
+ }
+
+ private ILSMMemoryComponent getOldestReadableMemoryComponent() {
+ synchronized (getOperationTracker()) {
+ int cmc = currentMutableComponentId.get();
+ int numImmutableMemoryComponents = Integer.min(numScheduledFlushes, memoryComponents.size());
+ int next = numScheduledFlushes < memoryComponents.size() ? cmc : getNextToBeFlushed();
+ for (int i = 0; i < numImmutableMemoryComponents; i++) {
+ next--;
+ if (next < 0) {
+ next = memoryComponents.size() - 1;
+ }
+ }
+
+ // start going forward
+ for (int i = 0; i < numImmutableMemoryComponents; i++) {
+ if (memoryComponents.get(next).isReadable()) {
+ return memoryComponents.get(next);
+ }
+ next++;
+ if (next == memoryComponents.size()) {
+ next = 0;
+ }
+ }
+ throw new IllegalStateException("Couldn't find any readable component");
+ }
+ }
+
+ private int getNextToBeFlushed() {
+ // we have:
+ // 1. currentMemeoryComponent
+ // 2. numMemoryComponents
+ // 3. numScheduledFlushes
+ int diff = numScheduledFlushes % memoryComponents.size();
+ int cmc = currentMutableComponentId.get() - diff;
+ return cmc < 0 ? memoryComponents.size() + cmc : cmc;
}
@Override
public final IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex) throws HyracksDataException {
+ return createBulkLoader(fillLevel, verifyInput, numElementsHint, checkIfEmptyIndex, null);
+ }
+
+ @Override
+ public IIndexBulkLoader createBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex, Map<String, Object> parameters) throws HyracksDataException {
if (checkIfEmptyIndex && !isEmptyIndex()) {
throw HyracksDataException.create(ErrorCode.LOAD_NON_EMPTY_INDEX);
}
- return createBulkLoader(fillLevel, verifyInput, numElementsHint);
+ return createBulkLoader(fillFactor, verifyInput, numElementsHint, parameters);
}
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpIndexAccessParameters.INSTANCE);
- opCtx.setIoOperationType(LSMIOOperationType.LOAD);
- ioOpCallback.beforeOperation(opCtx);
+ opCtx.setParameters(parameters);
+ LSMComponentFileReferences componentFileRefs = fileManager.getRelFlushFileReference();
+ LoadOperation loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+ loadOp.setNewComponent(createDiskComponent(bulkLoadComponentFactory,
+ componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true));
+ ioOpCallback.scheduled(loadOp);
+ opCtx.setIoOperation(loadOp);
return new LSMIndexDiskComponentBulkLoader(this, opCtx, fillLevel, verifyInput, numElementsHint);
}
@@ -503,7 +612,6 @@
@Override
public void changeMutableComponent() {
currentMutableComponentId.set((currentMutableComponentId.get() + 1) % memoryComponents.size());
- memoryComponents.get(currentMutableComponentId.get()).requestActivation();
}
@Override
@@ -527,11 +635,6 @@
}
@Override
- public ILSMIOOperationScheduler getIOScheduler() {
- return ioScheduler;
- }
-
- @Override
public ILSMIOOperationCallback getIOOperationCallback() {
return ioOpCallback;
}
@@ -556,8 +659,9 @@
@Override
public final String toString() {
return "{\"class\" : \"" + getClass().getSimpleName() + "\", \"dir\" : \"" + fileManager.getBaseDir()
- + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents.size()) + ", \"disk\" : "
- + diskComponents.size() + "}";
+ + "\", \"memory\" : " + (memoryComponents == null ? 0 : memoryComponents) + ", \"disk\" : "
+ + diskComponents.size() + ", \"num-scheduled-flushes\":" + numScheduledFlushes
+ + ", \"current-memory-component\":" + currentMutableComponentId.get() + "}";
}
@Override
@@ -567,8 +671,12 @@
@Override
public boolean isCurrentMutableComponentEmpty() throws HyracksDataException {
- //check if the current memory component has been modified
- return !memoryComponents.get(currentMutableComponentId.get()).isModified();
+ synchronized (getOperationTracker()) {
+ ILSMMemoryComponent cmc = getCurrentMemoryComponent();
+ ComponentState state = cmc.getState();
+ return state == ComponentState.READABLE_UNWRITABLE_FLUSHING || state == ComponentState.INACTIVE
+ || !cmc.isModified();
+ }
}
@Override
@@ -583,7 +691,7 @@
@Override
public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
- boolean bulkload, ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
+ ReplicationOperation operation, LSMOperationType opType) throws HyracksDataException {
//get set of files to be replicated for this component
Set<String> componentFiles = new HashSet<>();
@@ -593,7 +701,7 @@
}
ReplicationExecutionType executionType;
- if (bulkload) {
+ if (opType == LSMOperationType.LOAD) {
executionType = ReplicationExecutionType.SYNC;
} else {
executionType = ReplicationExecutionType.ASYNC;
@@ -700,40 +808,57 @@
}
@Override
+ public void resetCurrentComponentIndex() {
+ synchronized (lsmHarness.getOperationTracker()) {
+ // validate no reader in any of the memory components and that all of them are INVALID
+ for (ILSMMemoryComponent c : memoryComponents) {
+ if (c.getReaderCount() > 0) {
+ throw new IllegalStateException(
+ "Attempt to reset current component index while readers are inside the components. " + c);
+ }
+ if (c.getState() != ComponentState.INACTIVE) {
+ throw new IllegalStateException(
+ "Attempt to reset current component index while a component is not INACTIVE. " + c);
+ }
+ }
+ currentMutableComponentId.set(0);
+ memoryComponents.get(0);
+ try {
+ memoryComponents.get(0).resetId(null, true);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ }
+
+ @Override
public final ILSMDiskComponent flush(ILSMIOOperation operation) throws HyracksDataException {
ILSMIndexAccessor accessor = operation.getAccessor();
ILSMIndexOperationContext opCtx = accessor.getOpContext();
- if (opCtx.getOperation() == IndexOperation.DELETE_MEMORY_COMPONENT) {
+ ILSMMemoryComponent memoryComponent = (ILSMMemoryComponent) opCtx.getComponentHolder().get(0);
+ if (memoryComponent != getOldestReadableMemoryComponent()) {
+ throw new IllegalStateException("An attempt to flush a memory component that is not the oldest");
+ }
+ if (!memoryComponent.isModified() || opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS) {
return EmptyComponent.INSTANCE;
}
if (LOGGER.isInfoEnabled()) {
FlushOperation flushOp = (FlushOperation) operation;
- LOGGER.log(Level.INFO, "Flushing component with id: " + flushOp.getFlushingComponent().getId());
+ LOGGER.log(Level.INFO,
+ "Flushing component with id: " + flushOp.getFlushingComponent().getId() + " in the index " + this);
}
- try {
- return doFlush(operation);
- } catch (Exception e) {
- LOGGER.error("Fail to execute flush " + this, e);
- cleanUpFiles(operation, e);
- throw HyracksDataException.create(e);
- }
+ return doFlush(operation);
}
@Override
public final ILSMDiskComponent merge(ILSMIOOperation operation) throws HyracksDataException {
ILSMIndexAccessor accessor = operation.getAccessor();
ILSMIndexOperationContext opCtx = accessor.getOpContext();
- try {
- return opCtx.getOperation() == IndexOperation.DELETE_DISK_COMPONENTS ? EmptyComponent.INSTANCE
- : doMerge(operation);
- } catch (Exception e) {
- LOGGER.error("Fail to execute merge " + this, e);
- cleanUpFiles(operation, e);
- throw HyracksDataException.create(e);
- }
+ return opCtx.getOperation() == IndexOperation.DELETE_COMPONENTS ? EmptyComponent.INSTANCE : doMerge(operation);
}
- protected void cleanUpFiles(ILSMIOOperation operation, Exception e) {
+ @Override
+ public void cleanUpFilesForFailedOperation(ILSMIOOperation operation) {
LSMComponentFileReferences componentFiles = operation.getComponentFiles();
if (componentFiles == null) {
return;
@@ -742,14 +867,25 @@
for (FileReference file : files) {
try {
if (file != null) {
+ diskBufferCache.closeFileIfOpen(file);
diskBufferCache.deleteFile(file);
}
- } catch (HyracksDataException hde) {
- e.addSuppressed(hde);
+ } catch (Throwable th) { // NOSONAR Must catch all failures
+ operation.getFailure().addSuppressed(th);
}
}
}
+ @Override
+ public String getIndexIdentifier() {
+ return fileManager.getBaseDir().getAbsolutePath();
+ }
+
+ //Called when a memory component is reset
+ public void memoryComponentsReset() {
+ numScheduledFlushes = Integer.max(0, numScheduledFlushes - 1);
+ }
+
protected abstract LSMComponentFileReferences getMergeFileReferences(ILSMDiskComponent firstComponent,
ILSMDiskComponent lastComponent) throws HyracksDataException;
@@ -765,5 +901,4 @@
protected abstract ILSMDiskComponent doFlush(ILSMIOOperation operation) throws HyracksDataException;
protected abstract ILSMDiskComponent doMerge(ILSMIOOperation operation) throws HyracksDataException;
-
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
index c993874..2e133a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMIndexOperationContext.java
@@ -20,6 +20,7 @@
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -28,7 +29,7 @@
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
@@ -57,8 +58,8 @@
private long enterExitTime = 0L;
protected boolean skipFilter = false;
protected boolean recovery = false;
- private LSMIOOperationType ioOpType = LSMIOOperationType.NOOP;
- private ILSMDiskComponent newDiskComponent;
+ private ILSMIOOperation ioOperation;
+ private Map<String, Object> parametersMap;
public AbstractLSMIndexOperationContext(ILSMIndex index, int[] treeFields, int[] filterFields,
IBinaryComparatorFactory[] filterCmpFactories, ISearchOperationCallback searchCallback,
@@ -210,22 +211,23 @@
}
@Override
- public LSMIOOperationType getIoOperationType() {
- return ioOpType;
+ public ILSMIOOperation getIoOperation() {
+ return ioOperation;
}
@Override
- public void setIoOperationType(LSMIOOperationType ioOpType) {
- this.ioOpType = ioOpType;
+ public void setIoOperation(ILSMIOOperation ioOperation) {
+ this.ioOperation = ioOperation;
}
@Override
- public ILSMDiskComponent getNewComponent() {
- return newDiskComponent;
+ public void setParameters(Map<String, Object> map) {
+ this.parametersMap = map;
}
@Override
- public void setNewComponent(ILSMDiskComponent component) {
- this.newDiskComponent = component;
+ public Map<String, Object> getParameters() {
+ return parametersMap;
}
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
index 9596495..ab00dff 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMMemoryComponent.java
@@ -24,6 +24,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId.IdCompareResult;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -39,31 +40,56 @@
private final IVirtualBufferCache vbc;
private final AtomicBoolean isModified;
private int writerCount;
- private boolean requestedToBeActive;
+ private int pendingFlushes = 0;
private final MemoryComponentMetadata metadata;
private ILSMComponentId componentId;
- public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, boolean isActive,
- ILSMComponentFilter filter) {
+ public AbstractLSMMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
super(lsmIndex, filter);
this.vbc = vbc;
writerCount = 0;
- if (isActive) {
- state = ComponentState.READABLE_WRITABLE;
- } else {
- state = ComponentState.INACTIVE;
- }
+ state = ComponentState.INACTIVE;
isModified = new AtomicBoolean();
metadata = new MemoryComponentMetadata();
}
+ /**
+ * Prepare the component to be scheduled for an IO operation
+ *
+ * @param ioOperationType
+ * @throws HyracksDataException
+ */
+ @Override
+ public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+ activeate();
+ if (ioOperationType == LSMIOOperationType.FLUSH) {
+ if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
+ if (writerCount != 0) {
+ throw new IllegalStateException("Trying to schedule a flush when writerCount != 0");
+ }
+ state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
+ } else if (state == ComponentState.READABLE_UNWRITABLE_FLUSHING
+ || state == ComponentState.UNREADABLE_UNWRITABLE) {
+ // There is an ongoing flush. Increase pending flush count
+ pendingFlushes++;
+ } else {
+ throw new IllegalStateException("Trying to schedule a flush when the component state = " + state);
+ }
+ } else {
+ throw new UnsupportedOperationException("Unsupported operation " + ioOperationType);
+ }
+ }
+
+ private void activeate() throws HyracksDataException {
+ if (state == ComponentState.INACTIVE) {
+ state = ComponentState.READABLE_WRITABLE;
+ lsmIndex.getIOOperationCallback().recycled(this);
+ }
+ }
+
@Override
public boolean threadEnter(LSMOperationType opType, boolean isMutableComponent) throws HyracksDataException {
- if (state == ComponentState.INACTIVE && requestedToBeActive) {
- state = ComponentState.READABLE_WRITABLE;
- requestedToBeActive = false;
- lsmIndex.getIOOperationCallback().recycled(this, true);
- }
+ activeate();
switch (opType) {
case FORCE_MODIFICATION:
if (isMutableComponent) {
@@ -97,7 +123,6 @@
}
}
break;
- case REPLICATE:
case SEARCH:
if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE
|| state == ComponentState.READABLE_UNWRITABLE_FLUSHING) {
@@ -107,16 +132,18 @@
}
break;
case FLUSH:
- if (state == ComponentState.READABLE_WRITABLE || state == ComponentState.READABLE_UNWRITABLE) {
- if (writerCount != 0) {
- throw new IllegalStateException("Trying to flush when writerCount != 0");
- }
- state = ComponentState.READABLE_UNWRITABLE_FLUSHING;
- readerCount++;
- } else {
+ if (state == ComponentState.UNREADABLE_UNWRITABLE) {
return false;
}
- break;
+ if (state != ComponentState.READABLE_UNWRITABLE_FLUSHING) {
+ throw new IllegalStateException("Trying to flush when component state = " + state);
+ }
+ if (writerCount != 0) {
+ throw new IllegalStateException("Trying to flush when writerCount = " + writerCount);
+ }
+ readerCount++;
+ return true;
+
default:
throw new UnsupportedOperationException("Unsupported operation " + opType);
}
@@ -139,15 +166,14 @@
} else {
readerCount--;
if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
- state = ComponentState.INACTIVE;
+ reset();
}
}
break;
- case REPLICATE:
case SEARCH:
readerCount--;
if (state == ComponentState.UNREADABLE_UNWRITABLE && readerCount == 0) {
- state = ComponentState.INACTIVE;
+ reset();
}
break;
case FLUSH:
@@ -156,12 +182,12 @@
}
readerCount--;
if (failedOperation) {
- // if flush failed, return the component state to READABLE_UNWRITABLE
- state = ComponentState.READABLE_UNWRITABLE;
+ // If flush failed, keep the component state to READABLE_UNWRITABLE_FLUSHING
return;
}
+ // operation succeeded
if (readerCount == 0) {
- state = ComponentState.INACTIVE;
+ reset();
} else {
state = ComponentState.UNREADABLE_UNWRITABLE;
}
@@ -177,20 +203,15 @@
@Override
public boolean isReadable() {
- if (state == ComponentState.INACTIVE || state == ComponentState.UNREADABLE_UNWRITABLE) {
- return false;
+ return state != ComponentState.INACTIVE && state != ComponentState.UNREADABLE_UNWRITABLE;
+ }
+
+ @Override
+ public void setUnwritable() {
+ if (state != ComponentState.READABLE_WRITABLE) {
+ throw new IllegalStateException("Attempt to set unwritable a component that is " + state);
}
- return true;
- }
-
- @Override
- public void setState(ComponentState state) {
- this.state = state;
- }
-
- @Override
- public void requestActivation() {
- requestedToBeActive = true;
+ this.state = ComponentState.READABLE_UNWRITABLE;
}
@Override
@@ -210,12 +231,23 @@
@Override
public final void reset() throws HyracksDataException {
+ state = ComponentState.INACTIVE;
isModified.set(false);
metadata.reset();
if (filter != null) {
filter.reset();
}
doReset();
+ lsmIndex.memoryComponentsReset();
+ // a flush can be pending on a component that just completed its flush... here is when this can happen:
+ // primary index has 2 components, secondary index has 2 components.
+ // 2 flushes are scheduled on each p1, p2, s1, and s2.
+ // p1 and p2 both finish. primary component 1 gets full and secondary doesn't have any entries (optional field).
+ // then flush is scheduled on p1, s1 will have a pending flush in that case.
+ if (pendingFlushes > 0) {
+ schedule(LSMIOOperationType.FLUSH);
+ pendingFlushes--;
+ }
}
protected void doReset() throws HyracksDataException {
@@ -267,6 +299,7 @@
@Override
public final void deallocate() throws HyracksDataException {
try {
+ state = ComponentState.INACTIVE;
doDeallocate();
} finally {
getIndex().getBufferCache().close();
@@ -297,7 +330,7 @@
@Override
public void resetId(ILSMComponentId componentId, boolean force) throws HyracksDataException {
- if (!force && this.componentId != null && !componentId.missing() // for backward compatibility
+ if (!force && this.componentId != null
&& this.componentId.compareTo(componentId) != IdCompareResult.LESS_THAN) {
throw new IllegalStateException(
this + " receives illegal id. Old id " + this.componentId + ", new id " + componentId);
@@ -306,6 +339,15 @@
LOGGER.log(Level.INFO, "Component Id was reset from " + this.componentId + " to " + componentId);
}
this.componentId = componentId;
- LSMComponentIdUtils.persist(this.componentId, metadata);
+ if (componentId != null) {
+ LSMComponentIdUtils.persist(this.componentId, metadata);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "{\"class\":\"" + getClass().getSimpleName() + "\", \"state\":\"" + state + "\", \"writers\":"
+ + writerCount + ", \"readers\":" + readerCount + ", \"pendingFlushes\":" + pendingFlushes
+ + ", \"id\":\"" + componentId + "\"}";
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
new file mode 100644
index 0000000..373d7e7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractLSMWithBuddyMemoryComponent.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.impls;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.common.impls.AbstractTreeIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
+import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+
+public abstract class AbstractLSMWithBuddyMemoryComponent extends AbstractLSMMemoryComponent {
+
+ public AbstractLSMWithBuddyMemoryComponent(AbstractLSMIndex lsmIndex, IVirtualBufferCache vbc,
+ ILSMComponentFilter filter) {
+ super(lsmIndex, vbc, filter);
+ }
+
+ public abstract AbstractTreeIndex getBuddyIndex();
+
+ @Override
+ public void doReset() throws HyracksDataException {
+ super.doReset();
+ getBuddyIndex().deactivate();
+ getBuddyIndex().destroy();
+ getBuddyIndex().create();
+ getBuddyIndex().activate();
+ }
+
+ @Override
+ public void doAllocate() throws HyracksDataException {
+ super.doAllocate();
+ getBuddyIndex().create();
+ getBuddyIndex().activate();
+ }
+
+ @Override
+ public void doDeallocate() throws HyracksDataException {
+ super.doDeallocate();
+ getBuddyIndex().deactivate();
+ getBuddyIndex().destroy();
+ }
+
+ @Override
+ public void validate() throws HyracksDataException {
+ super.validate();
+ getBuddyIndex().validate();
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index a439ace..e4b845a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -18,94 +18,80 @@
*/
package org.apache.hyracks.storage.am.lsm.common.impls;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Deque;
import java.util.HashMap;
import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
-import java.util.concurrent.RunnableFuture;
-import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
-public class AsynchronousScheduler implements ILSMIOOperationScheduler {
+public class AsynchronousScheduler implements ILSMIOOperationScheduler, Closeable {
// Since this is a asynchronous scheduler, we make sure that flush operations coming from the same lsm index
// will be executed serially in same order of scheduling the operations. Look at asterix issue 630.
- public final static AsynchronousScheduler INSTANCE = new AsynchronousScheduler();
- private ExecutorService executor;
- private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<String, ILSMIOOperation>();
- private final Map<String, PriorityQueue<ILSMIOOperation>> waitingFlushOperations =
- new HashMap<String, PriorityQueue<ILSMIOOperation>>();
+ private final ExecutorService executor;
+ private final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
+ private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations = new HashMap<>();
+ private final Map<String, Throwable> failedGroups = new HashMap<>();
- public void init(ThreadFactory threadFactory) {
- // Creating an executor with the same configuration of Executors.newCachedThreadPool.
- executor = new ThreadPoolExecutor(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
- threadFactory) {
-
- @Override
- protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
- return new LSMIOOperationTask<T>(callable);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- protected void afterExecute(Runnable r, Throwable t) {
- super.afterExecute(r, t);
- LSMIOOperationTask<Boolean> task = (LSMIOOperationTask<Boolean>) r;
- ILSMIOOperation executedOp = task.getOperation();
- if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
- String id = executedOp.getIndexIdentifier();
- synchronized (this) {
- runningFlushOperations.remove(id);
- if (waitingFlushOperations.containsKey(id)) {
- try {
- ILSMIOOperation op = waitingFlushOperations.get(id).poll();
- if (op != null) {
- scheduleOperation(op);
- } else {
- waitingFlushOperations.remove(id);
- }
- } catch (HyracksDataException e) {
- t = e.getCause();
- }
- }
- }
- }
- }
- };
+ public AsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback) {
+ executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations,
+ waitingFlushOperations, failedGroups);
}
@Override
- public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
- if (operation.getIOOpertionType() == LSMIOOperationType.MERGE) {
- executor.submit(operation);
- } else if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
- String id = operation.getIndexIdentifier();
- synchronized (executor) {
- if (runningFlushOperations.containsKey(id)) {
- if (waitingFlushOperations.containsKey(id)) {
- waitingFlushOperations.get(id).offer(operation);
- } else {
- PriorityQueue<ILSMIOOperation> q = new PriorityQueue<ILSMIOOperation>();
- q.offer(operation);
- waitingFlushOperations.put(id, q);
- }
- } else {
- runningFlushOperations.put(id, operation);
- executor.submit(operation);
- }
- }
- } else {
- // this should never happen
- // just guard here to avoid silient failures in case of future extensions
- throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
+ public void scheduleOperation(ILSMIOOperation operation) {
+ switch (operation.getIOOpertionType()) {
+ case FLUSH:
+ scheduleFlush(operation);
+ break;
+ case MERGE:
+ executor.submit(operation);
+ break;
+ case NOOP:
+ return;
+ default:
+ // this should never happen
+ // just guard here to avoid silent failures in case of future extensions
+ throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
}
}
+
+ private void scheduleFlush(ILSMIOOperation operation) {
+ String id = operation.getIndexIdentifier();
+ synchronized (executor) {
+ if (failedGroups.containsKey(id)) {
+ // Group failure. Fail the operation right away
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
+ operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+ failedGroups.get(id)));
+ operation.complete();
+ return;
+ }
+ if (runningFlushOperations.containsKey(id)) {
+ if (waitingFlushOperations.containsKey(id)) {
+ waitingFlushOperations.get(id).offer(operation);
+ } else {
+ Deque<ILSMIOOperation> q = new ArrayDeque<>();
+ q.offer(operation);
+ waitingFlushOperations.put(id, q);
+ }
+ } else {
+ runningFlushOperations.put(id, operation);
+ executor.submit(operation);
+ }
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ executor.shutdown();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.java
deleted file mode 100644
index a8ee286..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/BlockingIOOperationCallbackWrapper.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.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-public class BlockingIOOperationCallbackWrapper implements ILSMIOOperationCallback {
-
- private boolean notified = false;
-
- private final ILSMIOOperationCallback wrappedCallback;
-
- public BlockingIOOperationCallbackWrapper(ILSMIOOperationCallback callback) {
- this.wrappedCallback = callback;
- }
-
- public synchronized void waitForIO() throws InterruptedException {
- while (!notified) {
- wait();
- }
- notified = false;
- }
-
- @Override
- public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- wrappedCallback.beforeOperation(opCtx);
- }
-
- @Override
- public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- wrappedCallback.afterOperation(opCtx);
- }
-
- @Override
- public synchronized void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- wrappedCallback.afterFinalize(opCtx);
- notifyAll();
- notified = true;
- }
-
- @Override
- public void recycled(ILSMMemoryComponent component, boolean componentSwitched) throws HyracksDataException {
- wrappedCallback.recycled(component, componentSwitched);
- }
-
- @Override
- public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
- wrappedCallback.allocated(component);
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
index f38614c..6e0606a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ChainedLSMDiskComponentBulkLoader.java
@@ -18,26 +18,32 @@
*/
package org.apache.hyracks.storage.am.lsm.common.impls;
-import java.util.LinkedList;
+import java.util.ArrayList;
import java.util.List;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.util.annotations.CriticalPath;
/**
* Class encapsulates a chain of operations, happening during an LSM disk component bulkload
*/
public class ChainedLSMDiskComponentBulkLoader implements ILSMDiskComponentBulkLoader {
- private List<IChainedComponentBulkLoader> bulkloaderChain = new LinkedList<>();
- private boolean isEmptyComponent = true;
- private boolean cleanedUpArtifacts = false;
+ private List<IChainedComponentBulkLoader> bulkloaderChain = new ArrayList<>();
+ private final ILSMIOOperation operation;
private final ILSMDiskComponent diskComponent;
private final boolean cleanupEmptyComponent;
+ private boolean isEmptyComponent = true;
+ private boolean cleanedUpArtifacts = false;
- public ChainedLSMDiskComponentBulkLoader(ILSMDiskComponent diskComponent, boolean cleanupEmptyComponent) {
+ public ChainedLSMDiskComponentBulkLoader(ILSMIOOperation operation, ILSMDiskComponent diskComponent,
+ boolean cleanupEmptyComponent) {
+ this.operation = operation;
this.diskComponent = diskComponent;
this.cleanupEmptyComponent = cleanupEmptyComponent;
}
@@ -46,14 +52,18 @@
bulkloaderChain.add(bulkloader);
}
+ @SuppressWarnings("squid:S1181")
@Override
+ @CriticalPath
public void add(ITupleReference tuple) throws HyracksDataException {
try {
ITupleReference t = tuple;
- for (IChainedComponentBulkLoader lsmBulkloader : bulkloaderChain) {
- t = lsmBulkloader.add(t);
+ final int bulkloadersCount = bulkloaderChain.size();
+ for (int i = 0; i < bulkloadersCount; i++) {
+ t = bulkloaderChain.get(i).add(t);
}
- } catch (Exception e) {
+ } catch (Throwable e) {
+ operation.setFailure(e);
cleanupArtifacts();
throw e;
}
@@ -62,14 +72,18 @@
}
}
+ @SuppressWarnings("squid:S1181")
@Override
+ @CriticalPath
public void delete(ITupleReference tuple) throws HyracksDataException {
try {
ITupleReference t = tuple;
- for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
- t = lsmOperation.delete(t);
+ final int bulkloadersCount = bulkloaderChain.size();
+ for (int i = 0; i < bulkloadersCount; i++) {
+ t = bulkloaderChain.get(i).delete(t);
}
- } catch (Exception e) {
+ } catch (Throwable e) {
+ operation.setFailure(e);
cleanupArtifacts();
throw e;
}
@@ -103,8 +117,14 @@
@Override
public void abort() throws HyracksDataException {
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
for (IChainedComponentBulkLoader lsmOperation : bulkloaderChain) {
lsmOperation.abort();
}
}
+
+ @Override
+ public ILSMIOOperation getOperation() {
+ return operation;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 7bb24dc..dad3bdc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -21,6 +21,7 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IExtendedModificationOperationCallback;
@@ -30,10 +31,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.common.IModificationOperationCallback;
import org.apache.hyracks.storage.common.ISearchOperationCallback;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -242,22 +242,22 @@
}
@Override
- public LSMIOOperationType getIoOperationType() {
+ public ILSMIOOperation getIoOperation() {
throw new UnsupportedOperationException();
}
@Override
- public void setIoOperationType(LSMIOOperationType ioOpType) {
+ public void setIoOperation(ILSMIOOperation ioOperation) {
throw new UnsupportedOperationException();
}
@Override
- public ILSMDiskComponent getNewComponent() {
+ public void setParameters(Map<String, Object> map) {
throw new UnsupportedOperationException();
}
@Override
- public void setNewComponent(ILSMDiskComponent component) {
+ public Map<String, Object> getParameters() {
throw new UnsupportedOperationException();
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index 13911ef..c642d82 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -49,10 +49,10 @@
IIndexAccessParameters iap =
new IndexAccessParameters(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
ILSMIndexAccessor accessor = index.createAccessor(iap);
- accessor.scheduleFullMerge(index.getIOOperationCallback());
+ accessor.scheduleFullMerge();
} else if (immutableComponents.size() >= numComponents) {
ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+ accessor.scheduleMerge(immutableComponents);
}
}
@@ -108,7 +108,7 @@
throw new IllegalStateException();
}
ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleMerge(index.getIOOperationCallback(), immutableComponents);
+ accessor.scheduleMerge(immutableComponents);
return true;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
index e3ca9f1..4c2ddb6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/EmptyComponent.java
@@ -27,6 +27,8 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
import org.apache.hyracks.storage.common.IIndex;
@@ -84,7 +86,7 @@
@Override
public ILSMComponentId getId() {
- return LSMComponentId.MISSING_COMPONENT_ID;
+ return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
}
@Override
@@ -144,9 +146,19 @@
}
@Override
- public ChainedLSMDiskComponentBulkLoader createBulkLoader(float fillFactor, boolean verifyInput,
- long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter, boolean cleanupEmptyComponent)
- throws HyracksDataException {
+ public ChainedLSMDiskComponentBulkLoader createBulkLoader(ILSMIOOperation operation, float fillFactor,
+ boolean verifyInput, long numElementsHint, boolean checkIfEmptyIndex, boolean withFilter,
+ boolean cleanupEmptyComponent) throws HyracksDataException {
return null;
}
+
+ @Override
+ public void schedule(LSMIOOperationType ioOperationType) throws HyracksDataException {
+ // Do nothing
+ }
+
+ @Override
+ public int getReaderCount() {
+ return 0;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
index 404c8e9..ab70ba1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ExternalIndexHarness.java
@@ -28,8 +28,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -39,15 +38,11 @@
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
import org.apache.hyracks.util.trace.ITracer;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
public class ExternalIndexHarness extends LSMHarness {
- private static final Logger LOGGER = LogManager.getLogger();
-
- public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- boolean replicationEnabled) {
- super(lsmIndex, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
+ public ExternalIndexHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, boolean replicationEnabled) {
+ super(lsmIndex, ioScheduler, mergePolicy, opTracker, replicationEnabled, ITracer.NONE);
}
@Override
@@ -104,14 +99,6 @@
}
ctx.setAccessingComponents(true);
}
- // Check if there is any action that is needed to be taken based on the operation type
- switch (opType) {
- case MERGE:
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- lsmIndex.getIOOperationCallback().beforeOperation(ctx);
- default:
- break;
- }
opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
return true;
}
@@ -135,7 +122,7 @@
if (replicationEnabled) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add((ILSMDiskComponent) c);
- lsmIndex.scheduleReplication(null, componentsToBeReplicated, false,
+ lsmIndex.scheduleReplication(null, componentsToBeReplicated,
ReplicationOperation.DELETE, opType);
}
((ILSMDiskComponent) c).deactivateAndDestroy();
@@ -155,7 +142,7 @@
if (replicationEnabled) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(newComponent);
- triggerReplication(componentsToBeReplicated, false, opType);
+ triggerReplication(componentsToBeReplicated, opType);
}
mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
}
@@ -206,54 +193,6 @@
}
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- callback.afterFinalize(ctx);
- return;
- }
- lsmIndex.scheduleMerge(ctx, callback);
- }
-
- @Override
- public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- fullMergeIsRequested.set(true);
- if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
- // If the merge cannot be scheduled because there is already an ongoing merge on subset/all of the components, then
- // whenever the current merge has finished, it will schedule the full merge again.
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- callback.afterFinalize(ctx);
- return;
- }
- fullMergeIsRequested.set(false);
- lsmIndex.scheduleMerge(ctx, callback);
- }
-
- @Override
- public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
- }
-
- ILSMDiskComponent newComponent = null;
- try {
- newComponent = lsmIndex.merge(operation);
- ctx.setNewComponent(newComponent);
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- operation.getCallback().afterOperation(ctx);
- newComponent.markAsValid(lsmIndex.isDurable());
- } finally {
- exitComponents(ctx, LSMOperationType.MERGE, newComponent, false);
- operation.getCallback().afterFinalize(ctx);
- }
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Finished the merge operation for index: " + lsmIndex);
- }
- }
-
- @Override
public void addBulkLoadedComponent(ILSMDiskComponent c) throws HyracksDataException {
c.markAsValid(lsmIndex.isDurable());
synchronized (opTracker) {
@@ -261,7 +200,7 @@
if (replicationEnabled) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(c);
- triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+ triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
}
// Enter the component
enterComponent(c);
@@ -304,14 +243,13 @@
}
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- ctx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.afterFinalize(ctx);
+ public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ return NoOpIoOperation.INSTANCE;
}
@Override
- public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+ public void flush(ILSMIOOperation operation) throws HyracksDataException {
+ throw new UnsupportedOperationException();
}
@Override
@@ -354,7 +292,7 @@
if (replicationEnabled) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(diskComponent);
- lsmIndex.scheduleReplication(null, componentsToBeReplicated, false, ReplicationOperation.DELETE, null);
+ lsmIndex.scheduleReplication(null, componentsToBeReplicated, ReplicationOperation.DELETE, null);
}
diskComponent.deactivateAndDestroy();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
index d835021..b2a2e48 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/FlushOperation.java
@@ -23,11 +23,10 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-public abstract class FlushOperation extends AbstractIoOperation implements Comparable<ILSMIOOperation> {
+public abstract class FlushOperation extends AbstractIoOperation {
public FlushOperation(ILSMIndexAccessor accessor, FileReference target, ILSMIOOperationCallback callback,
String indexIdentifier) {
@@ -35,9 +34,9 @@
}
@Override
- public Boolean call() throws HyracksDataException {
+ public LSMIOOperationStatus call() throws HyracksDataException {
accessor.flush(this);
- return true;
+ return getStatus();
}
@Override
@@ -65,14 +64,6 @@
}
@Override
- public int compareTo(ILSMIOOperation o) {
- if (o instanceof FlushOperation) {
- return target.getFile().getName().compareTo(((FlushOperation) o).getTarget().getFile().getName());
- }
- return -1;
- }
-
- @Override
public boolean equals(Object o) {
return (o instanceof FlushOperation)
&& Objects.equals(target.getFile().getName(), ((FlushOperation) o).target.getFile().getName());
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
new file mode 100644
index 0000000..354b1af
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.impls;
+
+import java.util.Deque;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.RunnableFuture;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+
+public class IoOperationExecutor extends ThreadPoolExecutor {
+
+ private final ILSMIOOperationScheduler scheduler;
+ private final IIoOperationFailedCallback callback;
+ private final Map<String, ILSMIOOperation> runningFlushOperations;
+ private final Map<String, Throwable> failedGroups;
+ private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations;
+
+ public IoOperationExecutor(ThreadFactory threadFactory, ILSMIOOperationScheduler scheduler,
+ IIoOperationFailedCallback callback, Map<String, ILSMIOOperation> runningFlushOperations,
+ Map<String, Deque<ILSMIOOperation>> waitingFlushOperations, Map<String, Throwable> failedGroups) {
+ super(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(), threadFactory);
+ this.scheduler = scheduler;
+ this.callback = callback;
+ this.runningFlushOperations = runningFlushOperations;
+ this.waitingFlushOperations = waitingFlushOperations;
+ this.failedGroups = failedGroups;
+ }
+
+ @Override
+ protected <T> RunnableFuture<T> newTaskFor(Callable<T> callable) {
+ return new LSMIOOperationTask<>(callable);
+ }
+
+ @Override
+ protected void afterExecute(Runnable r, Throwable t) {
+ super.afterExecute(r, t);
+ LSMIOOperationTask<?> task = (LSMIOOperationTask<?>) r;
+ ILSMIOOperation executedOp = task.getOperation();
+ try {
+ doAfterExecute(executedOp, t);
+ } catch (Throwable th) { // NOSONAR must catch all
+ callback.schedulerFailed(scheduler, th);
+ shutdown();
+ }
+ }
+
+ private void doAfterExecute(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+ final boolean failed = (t != null) || (executedOp.getStatus() == LSMIOOperationStatus.FAILURE);
+ if (failed) {
+ fail(executedOp, t != null ? t : executedOp.getFailure());
+ }
+ if (!failed || executedOp.getIOOpertionType() != LSMIOOperationType.FLUSH) {
+ executedOp.complete(); // destroy if merge or successful flush
+ }
+ if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ String id = executedOp.getIndexIdentifier();
+ synchronized (this) {
+ runningFlushOperations.remove(id);
+ if (waitingFlushOperations.containsKey(id)) {
+ ILSMIOOperation op = waitingFlushOperations.get(id).poll();
+ if (op != null) {
+ scheduler.scheduleOperation(op);
+ } else {
+ waitingFlushOperations.remove(id);
+ }
+ }
+ }
+ }
+ }
+
+ private void fail(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
+ callback.operationFailed(executedOp, t);
+ if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ executedOp.complete();
+ // Doesn't make sense to process further flush requests... Mark the operation group permanently failed
+ // Fail other scheduled operations
+ synchronized (this) {
+ String id = executedOp.getIndexIdentifier();
+ failedGroups.put(id, t);
+ runningFlushOperations.remove(id);
+ if (waitingFlushOperations.containsKey(id)) {
+ Deque<ILSMIOOperation> ops = waitingFlushOperations.remove(id);
+ ILSMIOOperation next = ops.poll();
+ while (next != null) {
+ next.setFailure(new RuntimeException("Operation group " + id + " has permanently failed", t));
+ next.setStatus(LSMIOOperationStatus.FAILURE);
+ next.complete();
+ next = ops.poll();
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
index 442af56..c7990a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentId.java
@@ -25,11 +25,11 @@
public static final long NOT_FOUND = -1;
- // Use to handle legacy datasets which do not have the component Id
- public static final ILSMComponentId MISSING_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
+ // Used to represent an empty index with no components
+ public static final LSMComponentId EMPTY_INDEX_LAST_COMPONENT_ID = new LSMComponentId(NOT_FOUND, NOT_FOUND);
// A default component id used for bulk loaded component
- public static final ILSMComponentId DEFAULT_COMPONENT_ID = new LSMComponentId(0, 0);
+ public static final LSMComponentId DEFAULT_COMPONENT_ID = new LSMComponentId(0, 0);
private long minId;
@@ -46,10 +46,12 @@
this.maxId = maxId;
}
+ @Override
public long getMinId() {
return this.minId;
}
+ @Override
public long getMaxId() {
return this.maxId;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
index e174153..3da57fd 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGenerator.java
@@ -28,18 +28,22 @@
*/
public class LSMComponentIdGenerator implements ILSMComponentIdGenerator {
+ private final int numComponents;
+ private int currentComponentIndex;
protected long previousTimestamp = -1L;
-
private ILSMComponentId componentId;
- public LSMComponentIdGenerator() {
+ public LSMComponentIdGenerator(int numComponents) {
+ this.numComponents = numComponents;
refresh();
+ currentComponentIndex = 0;
}
@Override
public void refresh() {
long ts = getCurrentTimestamp();
componentId = new LSMComponentId(ts, ts);
+ currentComponentIndex = (currentComponentIndex + 1) % numComponents;
}
@Override
@@ -47,6 +51,11 @@
return componentId;
}
+ @Override
+ public int getCurrentComponentIndex() {
+ return currentComponentIndex;
+ }
+
protected long getCurrentTimestamp() {
long timestamp = System.currentTimeMillis();
while (timestamp <= previousTimestamp) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
deleted file mode 100644
index d288ec8..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMComponentIdGeneratorFactory.java
+++ /dev/null
@@ -1,40 +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.hyracks.storage.am.lsm.common.impls;
-
-import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGenerator;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentIdGeneratorFactory;
-import org.apache.hyracks.storage.common.IResource;
-
-/**
- * A default implementation of {@link ILSMComponentIdGeneratorFactory}.
- *
- */
-public class LSMComponentIdGeneratorFactory implements ILSMComponentIdGeneratorFactory {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public ILSMComponentIdGenerator getComponentIdGenerator(INCServiceContext serviceCtx, IResource resource) {
- return new LSMComponentIdGenerator();
- }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
index 59f48d4..7a87a13 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMHarness.java
@@ -42,8 +42,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -51,9 +52,9 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
-import org.apache.hyracks.storage.am.lsm.common.util.IOOperationUtils;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.util.annotations.CriticalPath;
import org.apache.hyracks.util.trace.ITracer;
import org.apache.hyracks.util.trace.ITracer.Scope;
import org.apache.logging.log4j.Level;
@@ -64,6 +65,7 @@
private static final Logger LOGGER = LogManager.getLogger();
protected final ILSMIndex lsmIndex;
+ protected final ILSMIOOperationScheduler ioScheduler;
protected final ComponentReplacementContext componentReplacementCtx;
protected final ILSMMergePolicy mergePolicy;
protected final ILSMOperationTracker opTracker;
@@ -73,9 +75,10 @@
protected ITracer tracer;
protected long traceCategory;
- public LSMHarness(ILSMIndex lsmIndex, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
- boolean replicationEnabled, ITracer tracer) {
+ public LSMHarness(ILSMIndex lsmIndex, ILSMIOOperationScheduler ioScheduler, ILSMMergePolicy mergePolicy,
+ ILSMOperationTracker opTracker, boolean replicationEnabled, ITracer tracer) {
this.lsmIndex = lsmIndex;
+ this.ioScheduler = ioScheduler;
this.opTracker = opTracker;
this.mergePolicy = mergePolicy;
this.tracer = tracer;
@@ -100,58 +103,12 @@
synchronized (opTracker) {
while (true) {
lsmIndex.getOperationalComponents(ctx);
- // Before entering the components, prune those corner cases that indeed should not proceed.
- switch (opType) {
- case FLUSH:
- // if the lsm index does not have memory components allocated, then nothing to flush
- if (!lsmIndex.isMemoryComponentsAllocated()) {
- return false;
- }
- ILSMMemoryComponent flushingComponent =
- (ILSMMemoryComponent) ctx.getComponentHolder().get(0);
- if (!flushingComponent.isModified()) {
- recycle(flushingComponent);
- return false;
- }
- if (flushingComponent.getWriterCount() > 0) {
- /*
- * This case is a case where even though FLUSH log was flushed to disk
- * and scheduleFlush is triggered, the current in-memory component (whose state was
- * changed to READABLE_WRITABLE (RW) from READABLE_UNWRITABLE(RU) before FLUSH log
- * was written to log tail (which is memory buffer of log file) and then the state was
- * changed back to RW (as shown in the following scenario)) can have writers based on
- * the current code base/design. Thus, the writer count of the component may be greater
- * than 0. if this happens, intead of throwing exception, scheduleFlush() deal with
- * this situation by not flushing the component.
- * for more detailed information: ASTERIXDB-1027
- */
- return false;
- }
- break;
- case MERGE:
- if (ctx.getComponentHolder().size() < 2
- && ctx.getOperation() != IndexOperation.DELETE_DISK_COMPONENTS) {
- // There is only a single component. There is nothing to merge.
- return false;
- }
- break;
- default:
- break;
- }
if (enterComponents(ctx, opType)) {
return true;
} else if (isTryOperation) {
return false;
}
try {
- // Flush and merge operations should never reach this wait call,
- // because they are always try operations. If they fail to enter the components,
- // then it means that there are an ongoing flush/merge operation on
- // the same components, so they should not proceed.
- if (opType == LSMOperationType.MODIFICATION) {
- // before waiting, make sure the index is in a modifiable state to avoid waiting forever.
- ensureIndexModifiable();
- }
opTracker.wait();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -166,20 +123,7 @@
}
}
- private void recycle(ILSMMemoryComponent flushingComponent) throws HyracksDataException {
- if (flushingComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
- //The mutable component has not been modified by any writer.
- // There is nothing to flush. Since the component is empty, set its state back
- // to READABLE_WRITABLE only when it's state has been set to READABLE_UNWRITABLE
- flushingComponent.setState(ComponentState.READABLE_WRITABLE);
- opTracker.notifyAll(); // NOSONAR: Always synchronized from caller
- // Call recycled only when we change it's state is reset back to READABLE_WRITABLE
- // Otherwise, if the component is in other state, e.g., INACTIVE, or
- // READABLE_UNWRITABLE_FLUSHING, it's not considered as being recycled here.
- lsmIndex.getIOOperationCallback().recycled(flushingComponent, false);
- }
- }
-
+ @CriticalPath
protected boolean enterComponents(ILSMIndexOperationContext ctx, LSMOperationType opType)
throws HyracksDataException {
validateOperationEnterComponentsState(ctx);
@@ -187,9 +131,11 @@
int numEntered = 0;
boolean entranceSuccessful = false;
try {
- for (ILSMComponent c : components) {
- boolean isMutableComponent = numEntered == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
- if (!c.threadEnter(opType, isMutableComponent)) {
+ final int componentsCount = components.size();
+ for (int i = 0; i < componentsCount; i++) {
+ final ILSMComponent component = components.get(i);
+ boolean isMutableComponent = numEntered == 0 && component.getType() == LSMComponentType.MEMORY;
+ if (!component.threadEnter(opType, isMutableComponent)) {
break;
}
numEntered++;
@@ -202,14 +148,14 @@
throw e;
} finally {
if (!entranceSuccessful) {
- int i = 0;
- for (ILSMComponent c : components) {
+ final int componentsCount = components.size();
+ for (int i = 0; i < componentsCount; i++) {
+ final ILSMComponent component = components.get(i);
if (numEntered == 0) {
break;
}
- boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
- c.threadExit(opType, true, isMutableComponent);
- i++;
+ boolean isMutableComponent = i == 0 && component.getType() == LSMComponentType.MEMORY;
+ component.threadExit(opType, true, isMutableComponent);
numEntered--;
}
}
@@ -218,44 +164,26 @@
return false;
}
ctx.setAccessingComponents(true);
- // Check if there is any action that is needed to be taken based on the operation type
- switch (opType) {
- case FLUSH:
- ctx.setIoOperationType(LSMIOOperationType.FLUSH);
- lsmIndex.getIOOperationCallback().beforeOperation(ctx);
- // Changing the flush status should *always* precede changing the mutable component.
- lsmIndex.changeFlushStatusForCurrentMutableCompoent(false);
- lsmIndex.changeMutableComponent();
- // Notify all waiting threads whenever a flush has been scheduled since they will check
- // again if they can grab and enter the mutable component.
- opTracker.notifyAll(); // NOSONAR: Always called from a synchronized block
- break;
- case MERGE:
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- lsmIndex.getIOOperationCallback().beforeOperation(ctx);
- break;
- default:
- break;
- }
opTracker.beforeOperation(lsmIndex, opType, ctx.getSearchOperationCallback(), ctx.getModificationCallback());
return true;
}
+ @CriticalPath
private void doExitComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
ILSMDiskComponent newComponent, boolean failedOperation) throws HyracksDataException {
- /**
+ /*
* FLUSH and MERGE operations should always exit the components
* to notify waiting threads.
*/
if (!ctx.isAccessingComponents() && opType != LSMOperationType.FLUSH && opType != LSMOperationType.MERGE) {
return;
}
- List<ILSMDiskComponent> inactiveDiskComponents = null;
+ List<ILSMDiskComponent> inactiveDiskComponents;
List<ILSMDiskComponent> inactiveDiskComponentsToBeDeleted = null;
try {
synchronized (opTracker) {
try {
- /**
+ /*
* [flow control]
* If merge operations are lagged according to the merge policy,
* flushing in-memory components are hold until the merge operation catches up.
@@ -319,7 +247,7 @@
try {
//schedule a replication job to delete these inactive disk components from replicas
if (replicationEnabled) {
- lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted, false,
+ lsmIndex.scheduleReplication(null, inactiveDiskComponentsToBeDeleted,
ReplicationOperation.DELETE, opType);
}
for (ILSMDiskComponent c : inactiveDiskComponentsToBeDeleted) {
@@ -343,10 +271,12 @@
// newComponent is null if the flush op. was not performed.
if (!failedOperation && newComponent != null) {
lsmIndex.addDiskComponent(newComponent);
+ // TODO: The following should also replicate component Id
+ // even if empty component
if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(newComponent);
- triggerReplication(componentsToBeReplicated, false, opType);
+ triggerReplication(componentsToBeReplicated, opType);
}
mergePolicy.diskComponentAdded(lsmIndex, false);
}
@@ -358,7 +288,7 @@
if (replicationEnabled && newComponent != EmptyComponent.INSTANCE) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(newComponent);
- triggerReplication(componentsToBeReplicated, false, opType);
+ triggerReplication(componentsToBeReplicated, opType);
}
mergePolicy.diskComponentAdded(lsmIndex, fullMergeIsRequested.get());
}
@@ -368,13 +298,16 @@
}
}
+ @CriticalPath
private void exitOperationalComponents(ILSMIndexOperationContext ctx, LSMOperationType opType,
boolean failedOperation) throws HyracksDataException {
// First check if there is any action that is needed to be taken
// based on the state of each component.
- for (int i = 0; i < ctx.getComponentHolder().size(); i++) {
- ILSMComponent c = ctx.getComponentHolder().get(i);
- boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY ? true : false;
+ final List<ILSMComponent> componentHolder = ctx.getComponentHolder();
+ final int componentsCount = componentHolder.size();
+ for (int i = 0; i < componentsCount; i++) {
+ final ILSMComponent c = componentHolder.get(i);
+ boolean isMutableComponent = i == 0 && c.getType() == LSMComponentType.MEMORY;
c.threadExit(opType, failedOperation, isMutableComponent);
if (c.getType() == LSMComponentType.MEMORY) {
switch (c.getState()) {
@@ -386,7 +319,6 @@
break;
case INACTIVE:
tracer.instant(c.toString(), traceCategory, Scope.p, lsmIndex.toString());
- ((AbstractLSMMemoryComponent) c).reset();
// Notify all waiting threads whenever the mutable component's state
// has changed to inactive. This is important because even though we switched
// the mutable components, it is possible that the component that we just
@@ -548,129 +480,137 @@
}
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- if (!getAndEnterComponents(ctx, LSMOperationType.FLUSH, true)) {
- ctx.setIoOperationType(LSMIOOperationType.FLUSH);
- callback.afterFinalize(ctx);
- return;
+ public ILSMIOOperation scheduleFlush(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ ILSMIOOperation flush;
+ LOGGER.info("Flush is being scheduled on {}", lsmIndex);
+ synchronized (opTracker) {
+ if (!lsmIndex.isMemoryComponentsAllocated()) {
+ lsmIndex.allocateMemoryComponents();
+ }
+ try {
+ flush = lsmIndex.createFlushOperation(ctx);
+ } finally {
+ // Notify all waiting threads whenever a flush has been scheduled since they will check
+ // again if they can grab and enter the mutable component.
+ opTracker.notifyAll();
+ }
}
- lsmIndex.scheduleFlush(ctx, callback);
+ ioScheduler.scheduleOperation(flush);
+ return flush;
}
+ @SuppressWarnings("squid:S2142")
@Override
- public void flush(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
+ public void flush(ILSMIOOperation operation) throws HyracksDataException {
if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Started a flush operation for index: " + lsmIndex + " ...");
+ LOGGER.info("Started a flush operation for index: {}", lsmIndex);
+ }
+ synchronized (opTracker) {
+ while (!enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH)) {
+ try {
+ opTracker.wait();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ }
}
try {
- ILSMDiskComponent newComponent = null;
- boolean failedOperation = false;
- try {
- newComponent = lsmIndex.flush(operation);
- ctx.setNewComponent(newComponent);
- ctx.setIoOperationType(LSMIOOperationType.FLUSH);
- operation.getCallback().afterOperation(ctx);
- newComponent.markAsValid(lsmIndex.isDurable());
- } catch (Throwable e) { // NOSONAR Log and re-throw
- failedOperation = true;
- if (LOGGER.isErrorEnabled()) {
- LOGGER.log(Level.ERROR, "Flush failed on " + lsmIndex, e);
- }
- throw e;
- } finally {
- exitComponents(ctx, LSMOperationType.FLUSH, newComponent, failedOperation);
- ctx.setIoOperationType(LSMIOOperationType.FLUSH);
- operation.getCallback().afterFinalize(ctx);
-
- }
+ doIo(operation);
} finally {
- /*
- * Completion of flush/merge operations is done explicitly here to make sure all generated files during
- * io operations is completed before the io operation is declared complete
- */
- opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH, ctx.getSearchOperationCallback(),
- ctx.getModificationCallback());
+ exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.FLUSH, operation.getNewComponent(),
+ operation.getStatus() == LSMIOOperationStatus.FAILURE);
+ opTracker.completeOperation(lsmIndex, LSMOperationType.FLUSH,
+ operation.getAccessor().getOpContext().getSearchOperationCallback(),
+ operation.getAccessor().getOpContext().getModificationCallback());
}
if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Finished the flush operation for index: " + lsmIndex);
+ LOGGER.info("Finished the flush operation for index: {}. Result: ", lsmIndex, operation.getStatus());
+ }
+ }
+
+ public void doIo(ILSMIOOperation operation) {
+ try {
+ operation.getCallback().beforeOperation(operation);
+ ILSMDiskComponent newComponent = operation.getIOOpertionType() == LSMIOOperationType.FLUSH
+ ? lsmIndex.flush(operation) : lsmIndex.merge(operation);
+ operation.setNewComponent(newComponent);
+ operation.getCallback().afterOperation(operation);
+ if (newComponent != null) {
+ newComponent.markAsValid(lsmIndex.isDurable());
+ }
+ } catch (Throwable e) { // NOSONAR Must catch all
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
+ operation.setFailure(e);
+ if (LOGGER.isErrorEnabled()) {
+ LOGGER.log(Level.ERROR, "{} operation failed on {}", operation.getIOOpertionType(), lsmIndex, e);
+ }
+ } finally {
+ try {
+ operation.getCallback().afterFinalize(operation);
+ } catch (Throwable th) {// NOSONAR Must catch all
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
+ operation.setFailure(th);
+ if (LOGGER.isErrorEnabled()) {
+ LOGGER.log(Level.ERROR, "{} operation.afterFinalize failed on {}", operation.getIOOpertionType(),
+ lsmIndex, th);
+ }
+ }
+ }
+ // if the operation failed, we need to cleanup files
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ lsmIndex.cleanUpFilesForFailedOperation(operation);
}
}
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
- LOGGER.info("Failed to enter components for merge operation. Calling finalize");
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- callback.afterFinalize(ctx);
- return;
+ public void merge(ILSMIOOperation operation) throws HyracksDataException {
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Started a merge operation for index: {}", lsmIndex);
}
- lsmIndex.scheduleMerge(ctx, callback);
+ synchronized (opTracker) {
+ enterComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE);
+ }
+ try {
+ doIo(operation);
+ } finally {
+ exitComponents(operation.getAccessor().getOpContext(), LSMOperationType.MERGE, operation.getNewComponent(),
+ operation.getStatus() == LSMIOOperationStatus.FAILURE);
+ opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE,
+ operation.getAccessor().getOpContext().getSearchOperationCallback(),
+ operation.getAccessor().getOpContext().getModificationCallback());
+ }
+ if (LOGGER.isInfoEnabled()) {
+ LOGGER.info("Finished the merge operation for index: {}. Result: {}", lsmIndex, operation.getStatus());
+ }
}
@Override
- public void scheduleFullMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- fullMergeIsRequested.set(true);
- if (!getAndEnterComponents(ctx, LSMOperationType.MERGE, true)) {
+ public ILSMIOOperation scheduleMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ ILSMIOOperation operation;
+ synchronized (opTracker) {
+ operation = lsmIndex.createMergeOperation(ctx);
+ }
+ ioScheduler.scheduleOperation(operation);
+ return operation;
+ }
+
+ @Override
+ public ILSMIOOperation scheduleFullMerge(ILSMIndexOperationContext ctx) throws HyracksDataException {
+ ILSMIOOperation operation;
+ synchronized (opTracker) {
+ fullMergeIsRequested.set(true);
+ ctx.getComponentsToBeMerged().addAll(lsmIndex.getDiskComponents());
+ operation = lsmIndex.createMergeOperation(ctx);
+ if (operation != NoOpIoOperation.INSTANCE) {
+ fullMergeIsRequested.set(false);
+ }
// If the merge cannot be scheduled because there is already an ongoing merge on
// subset/all of the components, then whenever the current merge has finished,
// it will schedule the full merge again.
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- callback.afterFinalize(ctx);
- return;
}
- fullMergeIsRequested.set(false);
- lsmIndex.scheduleMerge(ctx, callback);
- }
-
- @Override
- public void merge(ILSMIndexOperationContext ctx, ILSMIOOperation operation) throws HyracksDataException {
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Started a merge operation for index: " + lsmIndex + " ...");
- }
- try {
- ILSMDiskComponent newComponent = null;
- boolean failedOperation = false;
- try {
- newComponent = lsmIndex.merge(operation);
- ctx.setNewComponent(newComponent);
- ctx.setIoOperationType(LSMIOOperationType.MERGE);
- operation.getCallback().afterOperation(ctx);
- newComponent.markAsValid(lsmIndex.isDurable());
- } catch (Throwable e) { // NOSONAR: Log and re-throw
- failedOperation = true;
- if (LOGGER.isErrorEnabled()) {
- LOGGER.log(Level.ERROR, "Failed merge operation on " + lsmIndex, e);
- }
- throw e;
- } finally {
- exitComponents(ctx, LSMOperationType.MERGE, newComponent, failedOperation);
- operation.getCallback().afterFinalize(ctx);
- }
- } finally {
- /*
- * Completion of the merge operation is called here to and not on afterOperation because
- * deletion of old components comes after afterOperation is called and the number of
- * io operation should not be decremented before the operation is complete to avoid
- * index destroy from competing with the merge on deletion of the files.
- * The order becomes:
- * 1. scheduleMerge
- * 2. enterComponents
- * 3. beforeOperation (increment the numOfIoOperations)
- * 4. merge
- * 5. exitComponents
- * 6. afterOperation (no op)
- * 7. delete components
- * 8. completeOperation (decrement the numOfIoOperations)
- */
- opTracker.completeOperation(lsmIndex, LSMOperationType.MERGE, ctx.getSearchOperationCallback(),
- ctx.getModificationCallback());
- }
- if (LOGGER.isInfoEnabled()) {
- LOGGER.info("Finished the merge operation for index: " + lsmIndex);
- }
+ ioScheduler.scheduleOperation(operation);
+ return operation;
}
@Override
@@ -681,7 +621,7 @@
if (replicationEnabled) {
componentsToBeReplicated.clear();
componentsToBeReplicated.add(c);
- triggerReplication(componentsToBeReplicated, true, LSMOperationType.MERGE);
+ triggerReplication(componentsToBeReplicated, LSMOperationType.LOAD);
}
mergePolicy.diskComponentAdded(lsmIndex, false);
}
@@ -692,20 +632,20 @@
return opTracker;
}
- protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+ protected void triggerReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
throws HyracksDataException {
ILSMIndexAccessor accessor = lsmIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleReplication(lsmComponents, bulkload, opType);
+ accessor.scheduleReplication(lsmComponents, opType);
}
@Override
public void scheduleReplication(ILSMIndexOperationContext ctx, List<ILSMDiskComponent> lsmComponents,
- boolean bulkload, LSMOperationType opType) throws HyracksDataException {
+ LSMOperationType opType) throws HyracksDataException {
//enter the LSM components to be replicated to prevent them from being deleted until they are replicated
if (!getAndEnterComponents(ctx, LSMOperationType.REPLICATE, false)) {
return;
}
- lsmIndex.scheduleReplication(ctx, lsmComponents, bulkload, ReplicationOperation.REPLICATE, opType);
+ lsmIndex.scheduleReplication(ctx, lsmComponents, ReplicationOperation.REPLICATE, opType);
}
@Override
@@ -771,36 +711,6 @@
}
}
- /***
- * Ensures the index is in a modifiable state (no failed flushes)
- *
- * @throws HyracksDataException
- * if the index is not in a modifiable state
- */
- private void ensureIndexModifiable() throws HyracksDataException {
- // if current memory component has a flush request, it means that flush didn't start for it
- if (lsmIndex.hasFlushRequestForCurrentMutableComponent()) {
- return;
- }
- // find if there is any memory component which is in a writable state or eventually will be in a writable state
- for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
- switch (memoryComponent.getState()) {
- case INACTIVE:
- // will be activated on next modification
- case UNREADABLE_UNWRITABLE:
- // flush completed successfully but readers are still inside
- case READABLE_WRITABLE:
- // writable
- case READABLE_UNWRITABLE_FLUSHING:
- // flush is ongoing
- return;
- default:
- // continue to the next component
- }
- }
- throw HyracksDataException.create(ErrorCode.CANNOT_MODIFY_INDEX_DISK_IS_FULL);
- }
-
/**
* Waits for any lagging merge operations to finish to avoid breaking
* the merge policy (i.e. adding a new disk component can make the
@@ -824,40 +734,42 @@
}
}
+ @SuppressWarnings("squid:S2142")
@Override
public void deleteComponents(ILSMIndexOperationContext ctx, Predicate<ILSMComponent> predicate)
throws HyracksDataException {
- BlockingIOOperationCallbackWrapper ioCallback =
- new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
boolean deleteMemoryComponent;
+ ILSMIOOperation ioOperation = null;
synchronized (opTracker) {
waitForFlushesAndMerges();
- ensureNoFailedFlush();
// We always start with the memory component
ILSMMemoryComponent memComponent = lsmIndex.getCurrentMemoryComponent();
deleteMemoryComponent = predicate.test(memComponent);
if (deleteMemoryComponent) {
// schedule a delete for flushed component
ctx.reset();
- ctx.setOperation(IndexOperation.DELETE_MEMORY_COMPONENT);
- // ScheduleFlush is actually a try operation
- scheduleFlush(ctx, ioCallback);
+ ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
+ ioOperation = scheduleFlush(ctx);
}
}
// Here, we are releasing the opTracker to allow other operations:
// (searches, delete flush we will schedule, delete merge we will schedule).
if (deleteMemoryComponent) {
- IOOperationUtils.waitForIoOperation(ioCallback);
+ try {
+ ioOperation.sync();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(ioOperation.getFailure());
+ }
}
ctx.reset();
- ioCallback = new BlockingIOOperationCallbackWrapper(lsmIndex.getIOOperationCallback());
- ctx.setOperation(IndexOperation.DELETE_DISK_COMPONENTS);
+ ctx.setOperation(IndexOperation.DELETE_COMPONENTS);
List<ILSMDiskComponent> toBeDeleted;
synchronized (opTracker) {
waitForFlushesAndMerges();
- // Ensure that current memory component is empty and that no failed flushes happened so far
- // This is a workaround until ASTERIXDB-2106 is fixed
- ensureNoFailedFlush();
List<ILSMDiskComponent> diskComponents = lsmIndex.getDiskComponents();
for (ILSMDiskComponent component : diskComponents) {
if (predicate.test(component)) {
@@ -869,9 +781,17 @@
}
toBeDeleted = new ArrayList<>(ctx.getComponentsToBeMerged());
// ScheduleMerge is actually a try operation
- scheduleMerge(ctx, ioCallback);
+ ioOperation = scheduleMerge(ctx);
}
- IOOperationUtils.waitForIoOperation(ioCallback);
+ try {
+ ioOperation.sync();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw HyracksDataException.create(e);
+ }
+ if (ioOperation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(ioOperation.getFailure());
+ }
synchronized (opTracker) {
// ensure that merge has succeeded
for (ILSMDiskComponent component : toBeDeleted) {
@@ -882,21 +802,6 @@
}
}
- /**
- * This can only be called in the steady state where:
- * 1. no scheduled flushes
- * 2. no incoming data
- *
- * @throws HyracksDataException
- */
- private void ensureNoFailedFlush() throws HyracksDataException {
- for (ILSMMemoryComponent memoryComponent : lsmIndex.getMemoryComponents()) {
- if (memoryComponent.getState() == ComponentState.READABLE_UNWRITABLE) {
- throw HyracksDataException.create(ErrorCode.A_FLUSH_OPERATION_HAS_FAILED);
- }
- }
- }
-
private void waitForFlushesAndMerges() throws HyracksDataException {
while (flushingOrMerging()) {
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
index 5e105a4..10074f9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexDiskComponentBulkLoader.java
@@ -22,62 +22,81 @@
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
public class LSMIndexDiskComponentBulkLoader implements IIndexBulkLoader {
private final AbstractLSMIndex lsmIndex;
private final ILSMDiskComponentBulkLoader componentBulkLoader;
- private ILSMIndexOperationContext opCtx;
+ private final ILSMIndexOperationContext opCtx;
public LSMIndexDiskComponentBulkLoader(AbstractLSMIndex lsmIndex, ILSMIndexOperationContext opCtx, float fillFactor,
boolean verifyInput, long numElementsHint) throws HyracksDataException {
this.lsmIndex = lsmIndex;
this.opCtx = opCtx;
- // Note that by using a flush target file name, we state that the
- // new bulk loaded component is "newer" than any other merged component.
- opCtx.setNewComponent(lsmIndex.createBulkLoadTarget());
- this.componentBulkLoader =
- opCtx.getNewComponent().createBulkLoader(fillFactor, verifyInput, numElementsHint, false, true, true);
+ this.componentBulkLoader = opCtx.getIoOperation().getNewComponent().createBulkLoader(opCtx.getIoOperation(),
+ fillFactor, verifyInput, numElementsHint, false, true, true);
}
public ILSMDiskComponent getComponent() {
- return opCtx.getNewComponent();
+ return opCtx.getIoOperation().getNewComponent();
}
+ @SuppressWarnings("squid:S1181")
@Override
public void add(ITupleReference tuple) throws HyracksDataException {
- componentBulkLoader.add(tuple);
+ try {
+ componentBulkLoader.add(tuple);
+ } catch (Throwable th) {
+ opCtx.getIoOperation().setFailure(th);
+ throw th;
+ }
}
+ @SuppressWarnings("squid:S1181")
public void delete(ITupleReference tuple) throws HyracksDataException {
- componentBulkLoader.delete(tuple);
+ try {
+ componentBulkLoader.delete(tuple);
+ } catch (Throwable th) {
+ opCtx.getIoOperation().setFailure(th);
+ throw th;
+ }
}
@Override
public void end() throws HyracksDataException {
try {
- componentBulkLoader.end();
- if (opCtx.getNewComponent().getComponentSize() > 0) {
- //TODO(amoudi): Ensure Bulk load follow the same lifecycle Other Operations (Flush, Merge, etc).
- //then after operation should be called from harness as well
- //https://issues.apache.org/jira/browse/ASTERIXDB-1764
- lsmIndex.getIOOperationCallback().afterOperation(opCtx);
- lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getNewComponent());
+ try {
+ lsmIndex.getIOOperationCallback().afterOperation(opCtx.getIoOperation());
+ componentBulkLoader.end();
+ } catch (Throwable th) { // NOSONAR Must not call afterFinalize without setting failure
+ opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
+ opCtx.getIoOperation().setFailure(th);
+ throw th;
+ } finally {
+ lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+ }
+ if (opCtx.getIoOperation().getStatus() == LSMIOOperationStatus.SUCCESS
+ && opCtx.getIoOperation().getNewComponent().getComponentSize() > 0) {
+ lsmIndex.getHarness().addBulkLoadedComponent(opCtx.getIoOperation().getNewComponent());
}
} finally {
- lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+ lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
}
}
@Override
public void abort() throws HyracksDataException {
+ opCtx.getIoOperation().setStatus(LSMIOOperationStatus.FAILURE);
try {
- componentBulkLoader.abort();
- opCtx.setNewComponent(null);
- lsmIndex.getIOOperationCallback().afterOperation(opCtx);
+ try {
+ componentBulkLoader.abort();
+ } finally {
+ lsmIndex.getIOOperationCallback().afterFinalize(opCtx.getIoOperation());
+ }
} finally {
- lsmIndex.getIOOperationCallback().afterFinalize(opCtx);
+ lsmIndex.getIOOperationCallback().completed(opCtx.getIoOperation());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index 445a005..12caec4 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -125,7 +125,6 @@
rangeCursors[i].close();
}
}
- rangeCursors = null;
} finally {
if (lsmHarness != null) {
lsmHarness.endSearch(opCtx);
@@ -247,9 +246,9 @@
}
}
- public class PriorityQueueElement {
+ public static class PriorityQueueElement {
private ITupleReference tuple;
- private int cursorIndex;
+ private final int cursorIndex;
public PriorityQueueElement(int cursorIndex) {
tuple = null;
@@ -269,7 +268,7 @@
}
}
- public class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
+ public static class PriorityQueueComparator implements Comparator<PriorityQueueElement> {
protected MultiComparator cmp;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
index 1548f86..8412b8c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMTreeIndexAccessor.java
@@ -34,7 +34,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -120,12 +119,12 @@
@Override
public void flush(ILSMIOOperation operation) throws HyracksDataException {
- lsmHarness.flush(ctx, operation);
+ lsmHarness.flush(operation);
}
@Override
public void merge(ILSMIOOperation operation) throws HyracksDataException {
- lsmHarness.merge(ctx, operation);
+ lsmHarness.merge(operation);
}
@Override
@@ -135,33 +134,32 @@
}
@Override
- public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+ public ILSMIOOperation scheduleFlush() throws HyracksDataException {
ctx.setOperation(IndexOperation.FLUSH);
- lsmHarness.scheduleFlush(ctx, callback);
+ return lsmHarness.scheduleFlush(ctx);
}
@Override
- public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
- throws HyracksDataException {
+ public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
ctx.setOperation(IndexOperation.MERGE);
ctx.getComponentsToBeMerged().clear();
ctx.getComponentsToBeMerged().addAll(components);
- lsmHarness.scheduleMerge(ctx, callback);
+ return lsmHarness.scheduleMerge(ctx);
}
@Override
- public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+ public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
throws HyracksDataException {
ctx.setOperation(IndexOperation.REPLICATE);
ctx.getComponentsToBeReplicated().clear();
ctx.getComponentsToBeReplicated().addAll(lsmComponents);
- lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+ lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
}
@Override
- public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+ public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
ctx.setOperation(IndexOperation.FULL_MERGE);
- lsmHarness.scheduleFullMerge(ctx, callback);
+ return lsmHarness.scheduleFullMerge(ctx);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
new file mode 100644
index 0000000..21c52d0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LoadOperation.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+
+public class LoadOperation extends AbstractIoOperation {
+
+ private final LSMComponentFileReferences fileReferences;
+ private final Map<String, Object> parameters;
+
+ public LoadOperation(LSMComponentFileReferences fileReferences, ILSMIOOperationCallback callback,
+ String indexIdentifier, Map<String, Object> parameters) {
+ super(null, fileReferences.getInsertIndexFileReference(), callback, indexIdentifier);
+ this.fileReferences = fileReferences;
+ this.parameters = parameters;
+ }
+
+ @Override
+ public final LSMIOOperationType getIOOpertionType() {
+ return LSMIOOperationType.LOAD;
+ }
+
+ @Override
+ public LSMIOOperationStatus call() throws HyracksDataException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public LSMComponentFileReferences getComponentFiles() {
+ return fileReferences;
+ }
+
+ @Override
+ public void sync() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Map<String, Object> getParameters() {
+ return parameters;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
index ec2305d..9d7c449 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MergeOperation.java
@@ -41,9 +41,9 @@
}
@Override
- public Boolean call() throws HyracksDataException {
+ public LSMIOOperationStatus call() throws HyracksDataException {
accessor.merge(this);
- return true;
+ return getStatus();
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
index 9b25471..9bd5007 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/MultitenantVirtualBufferCache.java
@@ -220,4 +220,8 @@
vbc.deleteFile(file);
}
+ @Override
+ public void closeFileIfOpen(FileReference fileRef) {
+ throw new UnsupportedOperationException();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
index 3432624..600a02d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIOOperationCallbackFactory.java
@@ -20,15 +20,20 @@
import org.apache.hyracks.api.application.INCServiceContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
import org.apache.hyracks.storage.common.IResource;
-public enum NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
- INSTANCE;
+public class NoOpIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static final NoOpIOOperationCallbackFactory INSTANCE = new NoOpIOOperationCallbackFactory();
+
+ private NoOpIOOperationCallbackFactory() {
+ }
@Override
public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) {
@@ -47,22 +52,7 @@
}
@Override
- public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // Do nothing.
- }
-
- @Override
- public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
+ public void recycled(ILSMMemoryComponent component) {
// Do nothing.
}
@@ -70,5 +60,35 @@
public void allocated(ILSMMemoryComponent component) {
// Do nothing.
}
+
+ @Override
+ public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+ // Do nothing.
+ }
+
+ @Override
+ public void completed(ILSMIOOperation operation) {
+ // Do nothing.
+ }
+ }
+
+ @Override
+ public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+ return 0;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
new file mode 100644
index 0000000..f57c4ef
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.common.impls;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
+
+public class NoOpIoOperation implements ILSMIOOperation {
+ public static final NoOpIoOperation INSTANCE = new NoOpIoOperation();
+
+ private NoOpIoOperation() {
+ }
+
+ @Override
+ public IODeviceHandle getDevice() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ILSMIOOperationCallback getCallback() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public String getIndexIdentifier() {
+ return NoOpIoOperation.class.getSimpleName();
+ }
+
+ @Override
+ public LSMIOOperationType getIOOpertionType() {
+ return LSMIOOperationType.NOOP;
+ }
+
+ @Override
+ public LSMIOOperationStatus call() throws HyracksDataException {
+ return LSMIOOperationStatus.SUCCESS;
+ }
+
+ @Override
+ public FileReference getTarget() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ILSMIndexAccessor getAccessor() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public LSMComponentFileReferences getComponentFiles() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public Throwable getFailure() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void setFailure(Throwable failure) {
+ // No Op
+ }
+
+ @Override
+ public LSMIOOperationStatus getStatus() {
+ return LSMIOOperationStatus.SUCCESS;
+ }
+
+ @Override
+ public void setStatus(LSMIOOperationStatus status) {
+ // No Op
+ }
+
+ @Override
+ public ILSMDiskComponent getNewComponent() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void setNewComponent(ILSMDiskComponent component) {
+ // No Op
+ }
+
+ @Override
+ public void complete() {
+ // No Op
+ }
+
+ @Override
+ public void sync() {
+ // No Op
+ }
+
+ @Override
+ public void addCompleteListener(IoOperationCompleteListener listener) {
+ listener.completed(this);
+ }
+
+ @Override
+ public Map<String, Object> getParameters() {
+ return null;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
new file mode 100644
index 0000000..3b2b2c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperationFailedCallback.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.common.impls;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class NoOpIoOperationFailedCallback implements IIoOperationFailedCallback {
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ public static final NoOpIoOperationFailedCallback INSTANCE = new NoOpIoOperationFailedCallback();
+
+ private NoOpIoOperationFailedCallback() {
+ }
+
+ @Override
+ public void operationFailed(ILSMIOOperation operation, Throwable t) {
+ LOGGER.error("Operation {} failed", operation, t);
+ }
+
+ @Override
+ public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+ LOGGER.error("IO Scheduler failed", failure);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
index 73b5d20..b571e58 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/PrefixMergePolicy.java
@@ -31,7 +31,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
public class PrefixMergePolicy implements ILSMMergePolicy {
@@ -59,8 +58,7 @@
}
if (fullMergeIsRequested) {
- ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFullMerge(index.getIOOperationCallback());
+ index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleFullMerge();
return;
}
@@ -244,8 +242,7 @@
// Reverse the components order back to its original order
Collections.reverse(mergableComponents);
- ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleMerge(index.getIOOperationCallback(), mergableComponents);
+ index.createAccessor(NoOpIndexAccessParameters.INSTANCE).scheduleMerge(mergableComponents);
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
deleted file mode 100644
index 2c16be0..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/StubIOOperationCallback.java
+++ /dev/null
@@ -1,70 +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.hyracks.storage.am.lsm.common.impls;
-
-import java.util.List;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
-
-/**
- * This class is for testing. It's basically a way to get the new/old component info from the
- * harness callback simply.
- */
-
-public class StubIOOperationCallback implements ILSMIOOperationCallback {
-
- private ILSMIndexOperationContext opCtx = null;
-
- @Override
- public void beforeOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // Not interested in this
- }
-
- @Override
- public void afterOperation(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- this.opCtx = opCtx;
- }
-
- @Override
- public void afterFinalize(ILSMIndexOperationContext opCtx) throws HyracksDataException {
- // Redundant info from after
- }
-
- public List<ILSMDiskComponent> getLastOldComponents() {
- return opCtx.getComponentsToBeMerged();
- }
-
- public ILSMDiskComponent getLastNewComponent() {
- return opCtx.getNewComponent();
- }
-
- @Override
- public void recycled(ILSMMemoryComponent component, boolean componentSwitched) {
- // Not interested in this
- }
-
- @Override
- public void allocated(ILSMMemoryComponent component) {
- // Not interested in this
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
index 506dcea..ae3b4e3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousScheduler.java
@@ -18,8 +18,13 @@
*/
package org.apache.hyracks.storage.am.lsm.common.impls;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -27,18 +32,76 @@
public class SynchronousScheduler implements ILSMIOOperationScheduler {
private static final Logger LOGGER = LogManager.getLogger();
- public static final SynchronousScheduler INSTANCE = new SynchronousScheduler();
+ private final Map<String, ILSMIOOperation> runningFlushOperations = new ConcurrentHashMap<>();
+ private final Map<String, Throwable> failedGroups = new ConcurrentHashMap<>();
+ private final IIoOperationFailedCallback failureCallback;
- private SynchronousScheduler() {
+ public SynchronousScheduler(IIoOperationFailedCallback failureCallback) {
+ this.failureCallback = failureCallback;
}
@Override
- public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
+ public void scheduleOperation(ILSMIOOperation operation) {
+ try {
+ before(operation);
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ return;
+ }
+ run(operation);
+ } catch (Throwable e) { // NOSONAR: Must catch them all
+ throw new IllegalStateException(e);
+ } finally {
+ after(operation);
+ }
+ }
+
+ private void run(ILSMIOOperation operation) {
try {
operation.call();
- } catch (Exception e) {
- LOGGER.log(Level.ERROR, "IO Operation failed", e);
- throw HyracksDataException.create(e);
+ } catch (Throwable th) { // NOSONAR Must catch all
+ LOGGER.log(Level.ERROR, "IO Operation failed", th);
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
+ operation.setFailure(th);
+ }
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ failureCallback.operationFailed(operation, operation.getFailure());
+ }
+ }
+
+ private void after(ILSMIOOperation operation) {
+ if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ synchronized (runningFlushOperations) {
+ runningFlushOperations.remove(operation.getIndexIdentifier());
+ if (operation.getStatus() == LSMIOOperationStatus.FAILURE) {
+ failedGroups.putIfAbsent(operation.getIndexIdentifier(), operation.getFailure());
+ }
+ operation.complete();
+ runningFlushOperations.notifyAll();
+ }
+ } else {
+ operation.complete();
+ }
+ }
+
+ private void before(ILSMIOOperation operation) throws InterruptedException {
+ String id = operation.getIndexIdentifier();
+ if (operation.getIOOpertionType() == LSMIOOperationType.FLUSH) {
+ synchronized (runningFlushOperations) {
+ while (true) {
+ if (failedGroups.containsKey(id)) {
+ operation.setStatus(LSMIOOperationStatus.FAILURE);
+ operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
+ failedGroups.get(id)));
+ return;
+ }
+ if (runningFlushOperations.containsKey(id)) {
+ runningFlushOperations.wait();
+ } else {
+ runningFlushOperations.put(id, operation);
+ break;
+ }
+ }
+ }
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
index 823db04..132d5cf 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/SynchronousSchedulerProvider.java
@@ -22,12 +22,16 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-public enum SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
- INSTANCE;
+public class SynchronousSchedulerProvider implements ILSMIOOperationSchedulerProvider {
+ private static final long serialVersionUID = 1L;
+ public static final SynchronousSchedulerProvider INSTANCE = new SynchronousSchedulerProvider();
+
+ private SynchronousSchedulerProvider() {
+ }
@Override
public ILSMIOOperationScheduler getIoScheduler(INCServiceContext ctx) {
- return SynchronousScheduler.INSTANCE;
+ return new SynchronousScheduler(NoOpIoOperationFailedCallback.INSTANCE);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
index a7fbe06..07eeb28 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ThreadCountingTracker.java
@@ -58,7 +58,7 @@
if (opType == LSMOperationType.MODIFICATION && threadRefCount.decrementAndGet() == 0
&& index.hasFlushRequestForCurrentMutableComponent()) {
ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- accessor.scheduleFlush(NoOpIOOperationCallbackFactory.INSTANCE.createIoOpCallback(index));
+ accessor.scheduleFlush();
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 7238f8e..f1172f3 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -19,12 +19,16 @@
package org.apache.hyracks.storage.am.lsm.common.impls;
+import java.util.Map;
+
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.api.io.IODeviceHandle;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.IoOperationCompleteListener;
import org.apache.hyracks.util.trace.ITracer;
import org.apache.hyracks.util.trace.ITracer.Scope;
import org.apache.hyracks.util.trace.TraceUtils;
@@ -53,10 +57,7 @@
if (tracer.isEnabled(traceCategory)) {
tracer.instant("schedule-" + ioOpName, traceCategory, Scope.p,
"{\"path\": \"" + ioOp.getTarget().getRelativePath() + "\"}");
- }
- if (tracer.isEnabled(traceCategory)) {
- return ioOp instanceof Comparable ? new ComparableTracedIOOperation(ioOp, tracer, traceCategory)
- : new TracedIOOperation(ioOp, tracer, traceCategory);
+ return new TracedIOOperation(ioOp, tracer, traceCategory);
}
return ioOp;
}
@@ -86,7 +87,7 @@
}
@Override
- public Boolean call() throws HyracksDataException {
+ public LSMIOOperationStatus call() throws HyracksDataException {
final String name = getTarget().getRelativePath();
final long tid = tracer.durationB(name, traceCategory, null);
try {
@@ -111,33 +112,54 @@
public LSMComponentFileReferences getComponentFiles() {
return ioOp.getComponentFiles();
}
+
+ @Override
+ public Throwable getFailure() {
+ return ioOp.getFailure();
+ }
+
+ @Override
+ public void setFailure(Throwable failure) {
+ ioOp.setFailure(failure);
+ }
+
+ @Override
+ public LSMIOOperationStatus getStatus() {
+ return ioOp.getStatus();
+ }
+
+ @Override
+ public void setStatus(LSMIOOperationStatus status) {
+ ioOp.setStatus(status);
+ }
+
+ @Override
+ public ILSMDiskComponent getNewComponent() {
+ return ioOp.getNewComponent();
+ }
+
+ @Override
+ public void setNewComponent(ILSMDiskComponent component) {
+ ioOp.setNewComponent(component);
+ }
+
+ @Override
+ public void complete() {
+ ioOp.complete();
+ }
+
+ @Override
+ public void sync() throws InterruptedException {
+ ioOp.sync();
+ }
+
+ @Override
+ public void addCompleteListener(IoOperationCompleteListener listener) {
+ ioOp.addCompleteListener(listener);
+ }
+
+ @Override
+ public Map<String, Object> getParameters() {
+ return ioOp.getParameters();
+ }
}
-
-class ComparableTracedIOOperation extends TracedIOOperation implements Comparable<ILSMIOOperation> {
-
- protected ComparableTracedIOOperation(ILSMIOOperation ioOp, ITracer trace, long traceCategory) {
- super(ioOp, trace, traceCategory);
- }
-
- @Override
- public int hashCode() {
- return this.ioOp.hashCode();
- }
-
- @Override
- public boolean equals(Object other) {
- return other instanceof ILSMIOOperation && compareTo((ILSMIOOperation) other) == 0;
- }
-
- @Override
- public int compareTo(ILSMIOOperation other) {
- final ILSMIOOperation myIoOp = this.ioOp;
- if (myIoOp instanceof Comparable && other instanceof ComparableTracedIOOperation) {
- return ((Comparable) myIoOp).compareTo(((ComparableTracedIOOperation) other).getIoOp());
- }
- LOGGER.warn("Comparing ioOps of type " + myIoOp.getClass().getSimpleName() + " and "
- + other.getClass().getSimpleName() + " in " + getClass().getSimpleName());
- return Integer.signum(hashCode() - other.hashCode());
- }
-
-}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
index ef9852d..e63a5b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java
@@ -462,4 +462,10 @@
map.put("open", open);
return map;
}
+
+ @Override
+ public void closeFileIfOpen(FileReference fileRef) {
+ throw new UnsupportedOperationException();
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
deleted file mode 100644
index 9bd873d..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/IOOperationUtils.java
+++ /dev/null
@@ -1,43 +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.hyracks.storage.am.lsm.common.util;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
-public class IOOperationUtils {
- private static final Logger LOGGER = LogManager.getLogger();
-
- private IOOperationUtils() {
- }
-
- public static void waitForIoOperation(BlockingIOOperationCallbackWrapper ioCallback) throws HyracksDataException {
- // Note that the following call assumes that the io operation has succeeded.
- try {
- ioCallback.waitForIO();
- } catch (InterruptedException e) {
- LOGGER.log(Level.WARN, "Operation has been interrupted. returning");
- Thread.currentThread().interrupt();
- throw HyracksDataException.create(e);
- }
- }
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
index 6d4b0a7..1779527 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/util/LSMComponentIdUtils.java
@@ -25,8 +25,11 @@
import org.apache.hyracks.storage.am.lsm.common.api.IComponentMetadata;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentId;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class LSMComponentIdUtils {
+ private static final Logger LOGGER = LogManager.getLogger();
private static final MutableArrayValueReference COMPONENT_ID_MIN_KEY =
new MutableArrayValueReference("Component_Id_Min".getBytes());
@@ -43,7 +46,9 @@
long minId = ComponentUtils.getLong(metadata, COMPONENT_ID_MIN_KEY, LSMComponentId.NOT_FOUND, buffer);
long maxId = ComponentUtils.getLong(metadata, COMPONENT_ID_MAX_KEY, LSMComponentId.NOT_FOUND, buffer);
if (minId == LSMComponentId.NOT_FOUND || maxId == LSMComponentId.NOT_FOUND) {
- return LSMComponentId.MISSING_COMPONENT_ID;
+ LOGGER.warn("Invalid component id {} was persisted to a component metadata",
+ LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID);
+ return LSMComponentId.EMPTY_INDEX_LAST_COMPONENT_ID;
} else {
return new LSMComponentId(minId, maxId);
}
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 4eb7728..f2fb9af 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
@@ -95,14 +95,14 @@
opTrackerProvider.getOperationTracker(serviceCtx, this), ioScheduler, ioOpCallbackFactory,
invertedIndexFields, filterTypeTraits, filterCmpFactories, filterFields,
filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
- metadataPageManagerFactory);
+ 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, invertedIndexFields, filterTypeTraits, filterCmpFactories,
filterFields, filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable,
- metadataPageManagerFactory);
+ metadataPageManagerFactory, serviceCtx.getTracer());
}
}
}
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 5fda514..22556af 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
@@ -59,7 +59,6 @@
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
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.common.impls.LSMIndexSearchCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndexAccessor;
@@ -100,10 +99,10 @@
IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+ int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable, ITracer tracer) throws HyracksDataException {
super(ioManager, virtualBufferCaches, diskBufferCache, fileManager, bloomFilterFalsePositiveRate, mergePolicy,
opTracker, ioScheduler, ioOpCallbackFactory, componentFactory, componentFactory, filterFrameFactory,
- filterManager, filterFields, durable, filterHelper, invertedIndexFields, ITracer.NONE);
+ filterManager, filterFields, durable, filterHelper, invertedIndexFields, tracer);
this.tokenizerFactory = tokenizerFactory;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
@@ -111,7 +110,6 @@
this.tokenCmpFactories = tokenCmpFactories;
this.filterFieldsForNonBulkLoadOps = filterFieldsForNonBulkLoadOps;
this.invertedIndexFieldsForNonBulkLoadOps = invertedIndexFieldsForNonBulkLoadOps;
-
int i = 0;
for (IVirtualBufferCache virtualBufferCache : virtualBufferCaches) {
InMemoryInvertedIndex memInvIndex =
@@ -120,9 +118,8 @@
BTreeUtils.createBTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
invListTypeTraits, invListCmpFactories, BTreeLeafFrameType.REGULAR_NSM,
ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_del_" + i), false);
- LSMInvertedIndexMemoryComponent mutableComponent =
- new LSMInvertedIndexMemoryComponent(this, memInvIndex, deleteKeysBTree, virtualBufferCache,
- i == 0 ? true : false, filterHelper == null ? null : filterHelper.createFilter());
+ LSMInvertedIndexMemoryComponent mutableComponent = new LSMInvertedIndexMemoryComponent(this, memInvIndex,
+ deleteKeysBTree, virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
memoryComponents.add(mutableComponent);
++i;
}
@@ -296,7 +293,7 @@
}
ILSMDiskComponentBulkLoader componentBulkLoader =
- component.createBulkLoader(1.0f, false, numBTreeTuples, false, false, false);
+ component.createBulkLoader(operation, 1.0f, false, numBTreeTuples, false, false, false);
// Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index.
IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false);
@@ -349,65 +346,66 @@
LSMInvertedIndexMergeOperation mergeOp = (LSMInvertedIndexMergeOperation) operation;
RangePredicate mergePred = new RangePredicate(null, null, true, true, null, null);
IIndexCursor cursor = mergeOp.getCursor();
- try {
- ILSMIndexOperationContext opCtx = ((LSMIndexSearchCursor) cursor).getOpCtx();
- // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
- // Create an inverted index instance.
- ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
- mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
- ILSMDiskComponentBulkLoader componentBulkLoader;
- // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
- // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
- if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
- .get(diskComponents.size() - 1)) {
- // Keep the deleted tuples since the oldest disk component is not included in the merge operation
- LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
- new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
- try {
- long numElements = 0L;
- for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
- numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
- .getBloomFilter().getNumElements();
- }
- componentBulkLoader = component.createBulkLoader(1.0f, false, numElements, false, false, false);
- loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
- } finally {
- btreeCursor.destroy();
- }
- } else {
- componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
- }
- search(opCtx, cursor, mergePred);
+ ILSMIndexOperationContext opCtx = ((LSMInvertedIndexMergeCursor) cursor).getOpCtx();
+ // Scan diskInvertedIndexes ignoring the memoryInvertedIndex.
+ // Create an inverted index instance.
+ ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(),
+ mergeOp.getDeletedKeysBTreeTarget(), mergeOp.getBloomFilterTarget(), true);
+ ILSMDiskComponentBulkLoader componentBulkLoader;
+ // In case we must keep the deleted-keys BTrees, then they must be merged *before* merging the inverted
+ // indexes so that lsmHarness.endSearch() is called once when the inverted indexes have been merged.
+ if (mergeOp.getMergingComponents().get(mergeOp.getMergingComponents().size() - 1) != diskComponents
+ .get(diskComponents.size() - 1)) {
+ // Keep the deleted tuples since the oldest disk component is not included in the merge operation
+ LSMInvertedIndexDeletedKeysBTreeMergeCursor btreeCursor =
+ new LSMInvertedIndexDeletedKeysBTreeMergeCursor(opCtx);
try {
- while (cursor.hasNext()) {
- cursor.next();
- ITupleReference tuple = cursor.getTuple();
- componentBulkLoader.add(tuple);
- }
- } finally {
- cursor.close();
- }
- if (component.getLSMComponentFilter() != null) {
- List<ITupleReference> filterTuples = new ArrayList<>();
+ long numElements = 0L;
for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
- ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
- ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
- if (min != null) {
- filterTuples.add(min);
- }
- if (max != null) {
- filterTuples.add(max);
- }
+ numElements += ((LSMInvertedIndexDiskComponent) mergeOp.getMergingComponents().get(i))
+ .getBloomFilter().getNumElements();
}
- getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
- NoOpOperationCallback.INSTANCE);
- getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+ componentBulkLoader =
+ component.createBulkLoader(operation, 1.0f, false, numElements, false, false, false);
+ loadDeleteTuples(opCtx, btreeCursor, mergePred, componentBulkLoader);
+ } finally {
+ btreeCursor.destroy();
}
- componentBulkLoader.end();
- return component;
- } finally {
- cursor.close();
+ } else {
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
}
+ search(opCtx, cursor, mergePred);
+ try {
+ while (cursor.hasNext()) {
+ cursor.next();
+ componentBulkLoader.add(cursor.getTuple());
+ }
+ } finally {
+ try {
+ cursor.close();
+ } finally {
+ cursor.destroy();
+ }
+ }
+ if (component.getLSMComponentFilter() != null) {
+ List<ITupleReference> filterTuples = new ArrayList<>();
+ for (int i = 0; i < mergeOp.getMergingComponents().size(); ++i) {
+ ITupleReference min = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMinTuple();
+ ITupleReference max = mergeOp.getMergingComponents().get(i).getLSMComponentFilter().getMaxTuple();
+ if (min != null) {
+ filterTuples.add(min);
+ }
+ if (max != null) {
+ filterTuples.add(max);
+ }
+ }
+ getFilterManager().updateFilter(component.getLSMComponentFilter(), filterTuples,
+ NoOpOperationCallback.INSTANCE);
+ getFilterManager().writeFilter(component.getLSMComponentFilter(), component.getMetadataHolder());
+ }
+ componentBulkLoader.end();
+
+ return component;
}
private void loadDeleteTuples(ILSMIndexOperationContext opCtx,
@@ -488,16 +486,16 @@
throws HyracksDataException {
return new LSMInvertedIndexFlushOperation(new LSMInvertedIndexAccessor(getHarness(), opCtx),
componentFileRefs.getInsertIndexFileReference(), componentFileRefs.getDeleteIndexFileReference(),
- componentFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
+ componentFileRefs.getBloomFilterFileReference(), callback, getIndexIdentifier());
}
@Override
protected ILSMIOOperation createMergeOperation(AbstractLSMIndexOperationContext opCtx,
LSMComponentFileReferences mergeFileRefs, ILSMIOOperationCallback callback) throws HyracksDataException {
ILSMIndexAccessor accessor = new LSMInvertedIndexAccessor(getHarness(), opCtx);
- IIndexCursor cursor = new LSMInvertedIndexRangeSearchCursor(opCtx);
+ IIndexCursor cursor = new LSMInvertedIndexMergeCursor(opCtx);
return new LSMInvertedIndexMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
- fileManager.getBaseDir().getAbsolutePath());
+ getIndexIdentifier());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
index 247e44c..8e39b62 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java
@@ -30,7 +30,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
import org.apache.hyracks.storage.am.lsm.common.api.LSMOperationType;
@@ -86,43 +85,42 @@
}
@Override
- public void scheduleFlush(ILSMIOOperationCallback callback) throws HyracksDataException {
+ public ILSMIOOperation scheduleFlush() throws HyracksDataException {
ctx.setOperation(IndexOperation.FLUSH);
- lsmHarness.scheduleFlush(ctx, callback);
+ return lsmHarness.scheduleFlush(ctx);
}
@Override
public void flush(ILSMIOOperation operation) throws HyracksDataException {
- lsmHarness.flush(ctx, operation);
+ lsmHarness.flush(operation);
}
@Override
- public void scheduleMerge(ILSMIOOperationCallback callback, List<ILSMDiskComponent> components)
- throws HyracksDataException {
+ public ILSMIOOperation scheduleMerge(List<ILSMDiskComponent> components) throws HyracksDataException {
ctx.setOperation(IndexOperation.MERGE);
ctx.getComponentsToBeMerged().clear();
ctx.getComponentsToBeMerged().addAll(components);
- lsmHarness.scheduleMerge(ctx, callback);
+ return lsmHarness.scheduleMerge(ctx);
}
@Override
- public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, boolean bulkload, LSMOperationType opType)
+ public void scheduleReplication(List<ILSMDiskComponent> lsmComponents, LSMOperationType opType)
throws HyracksDataException {
ctx.setOperation(IndexOperation.REPLICATE);
ctx.getComponentsToBeReplicated().clear();
ctx.getComponentsToBeReplicated().addAll(lsmComponents);
- lsmHarness.scheduleReplication(ctx, lsmComponents, bulkload, opType);
+ lsmHarness.scheduleReplication(ctx, lsmComponents, opType);
}
@Override
- public void scheduleFullMerge(ILSMIOOperationCallback callback) throws HyracksDataException {
+ public ILSMIOOperation scheduleFullMerge() throws HyracksDataException {
ctx.setOperation(IndexOperation.FULL_MERGE);
- lsmHarness.scheduleFullMerge(ctx, callback);
+ return lsmHarness.scheduleFullMerge(ctx);
}
@Override
public void merge(ILSMIOOperation operation) throws HyracksDataException {
- lsmHarness.merge(ctx, operation);
+ lsmHarness.merge(operation);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
index f1f5241..faa90eb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java
@@ -60,7 +60,6 @@
ArrayList<IIndexAccessor> btreeAccessors = lsmInitialState.getDeletedKeysBTreeAccessors();
for (int i = 0; i < numBTrees; i++) {
rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(false);
- btreeAccessors.get(i).search(rangeCursors[i], btreePredicate);
}
IndexCursorUtils.open(btreeAccessors, rangeCursors, btreePredicate);
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
index 279a518..b030e83 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDiskComponent.java
@@ -25,11 +25,14 @@
import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
import org.apache.hyracks.storage.am.btree.impls.BTree;
import org.apache.hyracks.storage.am.common.api.IMetadataPageManager;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.impls.IChainedComponentBulkLoader;
+import org.apache.hyracks.storage.am.lsm.common.impls.IndexWithBuddyBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.util.ComponentUtils;
import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
+import org.apache.hyracks.storage.common.IIndexBulkLoader;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
public class LSMInvertedIndexDiskComponent extends AbstractLSMWithBuddyDiskComponent {
@@ -109,4 +112,14 @@
// Flush deleted keys BTree.
ComponentUtils.markAsValid(getBuddyIndex(), persist);
}
+
+ @Override
+ protected IChainedComponentBulkLoader createMergeIndexBulkLoader(float fillFactor, boolean verifyInput,
+ long numElementsHint, boolean checkIfEmptyIndex) throws HyracksDataException {
+ IIndexBulkLoader indexBulkLoader =
+ invIndex.createMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ IIndexBulkLoader buddyBulkLoader =
+ getBuddyIndex().createBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex);
+ return new IndexWithBuddyBulkLoader(indexBulkLoader, buddyBulkLoader);
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
index 77c62b4..a8005bc 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMemoryComponent.java
@@ -20,9 +20,9 @@
package org.apache.hyracks.storage.am.lsm.invertedindex.impls;
import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
import org.apache.hyracks.storage.am.lsm.invertedindex.inmemory.InMemoryInvertedIndex;
public class LSMInvertedIndexMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -31,8 +31,8 @@
private final BTree deletedKeysBTree;
public LSMInvertedIndexMemoryComponent(LSMInvertedIndex lsmIndex, InMemoryInvertedIndex invIndex,
- BTree deletedKeysBTree, IVirtualBufferCache vbc, boolean isActive, ILSMComponentFilter filter) {
- super(lsmIndex, vbc, isActive, filter);
+ BTree deletedKeysBTree, IVirtualBufferCache vbc, ILSMComponentFilter filter) {
+ super(lsmIndex, vbc, filter);
this.invIndex = invIndex;
this.deletedKeysBTree = deletedKeysBTree;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
new file mode 100644
index 0000000..c80455d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexMergeCursor.java
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.impls;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+import org.apache.hyracks.dataflow.common.utils.TupleUtils;
+import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilter;
+import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
+import org.apache.hyracks.storage.am.common.api.ILSMIndexCursor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueComparator;
+import org.apache.hyracks.storage.am.lsm.common.impls.LSMIndexSearchCursor.PriorityQueueElement;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndex;
+import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndexRangeSearchCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexAccessor;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.apache.hyracks.storage.common.MultiComparator;
+
+/**
+ * This cursor is specially designed and optimized for merging inverted index.
+ * For simplicity, it assumes all components are disk components, and the cursor is not reused.
+ *
+ */
+public class LSMInvertedIndexMergeCursor extends EnforcedIndexCursor implements ILSMIndexCursor {
+ protected final LSMInvertedIndexOpContext opCtx;
+ protected PriorityQueueElement outputTokenElement;
+ protected OnDiskInvertedIndexRangeSearchCursor[] rangeCursors;
+ protected PriorityQueueElement[] tokenQueueElements;
+ protected PriorityQueue<PriorityQueueElement> tokenQueue;
+ protected PriorityQueueComparator tokenQueueCmp;
+
+ protected PriorityQueueElement outputKeyElement;
+ protected PriorityQueueElement[] keyQueueElements;
+ protected PriorityQueue<PriorityQueueElement> keyQueue;
+ protected PriorityQueueComparator keyQueueCmp;
+
+ protected boolean needPushElementIntoKeyQueue;
+
+ protected ILSMHarness lsmHarness;
+
+ protected MultiComparator tokenCmp;
+ protected MultiComparator keyCmp;
+
+ protected List<ILSMComponent> operationalComponents;
+
+ // Assuming the cursor for all deleted-keys indexes are of the same type.
+ protected IIndexCursor[] deletedKeysBTreeCursors;
+ protected BloomFilter[] bloomFilters;
+ protected final long[] hashes = BloomFilter.createHashArray();
+ protected ArrayList<IIndexAccessor> deletedKeysBTreeAccessors;
+ protected RangePredicate deletedKeyBTreeSearchPred;
+
+ protected final TokenKeyPairTuple outputTuple;
+
+ public LSMInvertedIndexMergeCursor(ILSMIndexOperationContext opCtx) {
+ this.opCtx = (LSMInvertedIndexOpContext) opCtx;
+ outputTokenElement = null;
+ outputKeyElement = null;
+ needPushElementIntoKeyQueue = false;
+
+ IInvertedIndex invertedIndex = (IInvertedIndex) this.opCtx.getIndex();
+ this.outputTuple = new TokenKeyPairTuple(invertedIndex.getTokenTypeTraits().length,
+ invertedIndex.getInvListTypeTraits().length);
+
+ this.tokenCmp = MultiComparator.create(invertedIndex.getTokenCmpFactories());
+ this.keyCmp = MultiComparator.create(invertedIndex.getInvListCmpFactories());
+ this.tokenQueueCmp = new PriorityQueueComparator(tokenCmp);
+ this.keyQueueCmp = new PriorityQueueComparator(keyCmp);
+ }
+
+ public LSMInvertedIndexOpContext getOpCtx() {
+ return opCtx;
+ }
+
+ @Override
+ public void doOpen(ICursorInitialState initState, ISearchPredicate searchPred) throws HyracksDataException {
+ LSMInvertedIndexRangeSearchCursorInitialState lsmInitState =
+ (LSMInvertedIndexRangeSearchCursorInitialState) initState;
+ int numComponents = lsmInitState.getNumComponents();
+ rangeCursors = new OnDiskInvertedIndexRangeSearchCursor[numComponents];
+ for (int i = 0; i < numComponents; i++) {
+ IInvertedIndexAccessor invIndexAccessor = (IInvertedIndexAccessor) lsmInitState.getIndexAccessors().get(i);
+ rangeCursors[i] = (OnDiskInvertedIndexRangeSearchCursor) invIndexAccessor.createRangeSearchCursor();
+ invIndexAccessor.rangeSearch(rangeCursors[i], lsmInitState.getSearchPredicate());
+ }
+ lsmHarness = lsmInitState.getLSMHarness();
+ operationalComponents = lsmInitState.getOperationalComponents();
+ deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
+ bloomFilters = new BloomFilter[deletedKeysBTreeAccessors.size()];
+ if (!deletedKeysBTreeAccessors.isEmpty()) {
+ deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
+ for (int i = 0; i < operationalComponents.size(); i++) {
+ ILSMComponent component = operationalComponents.get(i);
+ deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false);
+ if (component.getType() == LSMComponentType.MEMORY) {
+ // No need for a bloom filter for the in-memory BTree.
+ bloomFilters[i] = null;
+ } else {
+ bloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
+ }
+ }
+ }
+ deletedKeyBTreeSearchPred = new RangePredicate(null, null, true, true, keyCmp, keyCmp);
+ initPriorityQueues();
+ }
+
+ private void initPriorityQueues() throws HyracksDataException {
+ int pqInitSize = (rangeCursors.length > 0) ? rangeCursors.length : 1;
+ tokenQueue = new PriorityQueue<>(pqInitSize, tokenQueueCmp);
+ keyQueue = new PriorityQueue<>(pqInitSize, keyQueueCmp);
+ tokenQueueElements = new PriorityQueueElement[pqInitSize];
+ keyQueueElements = new PriorityQueueElement[pqInitSize];
+ for (int i = 0; i < pqInitSize; i++) {
+ tokenQueueElements[i] = new PriorityQueueElement(i);
+ keyQueueElements[i] = new PriorityQueueElement(i);
+ }
+ for (int i = 0; i < rangeCursors.length; i++) {
+ if (rangeCursors[i].hasNext()) {
+ rangeCursors[i].next();
+ tokenQueueElements[i].reset(rangeCursors[i].getTuple());
+ tokenQueue.offer(tokenQueueElements[i]);
+ } else {
+ rangeCursors[i].close();
+ }
+ }
+ searchNextToken();
+ }
+
+ private void searchNextToken() throws HyracksDataException {
+ if (tokenQueue.isEmpty()) {
+ return;
+ }
+ if (!keyQueue.isEmpty()) {
+ throw new IllegalStateException("Illegal call of initializing key queue");
+ }
+ outputTokenElement = tokenQueue.poll();
+ initPushIntoKeyQueue(outputTokenElement);
+ ITupleReference tokenTuple = getTokenTuple(outputTokenElement);
+ outputTuple.setTokenTuple(tokenTuple);
+ // pop all same tokens
+ while (!tokenQueue.isEmpty()) {
+ PriorityQueueElement tokenElement = tokenQueue.peek();
+ if (TupleUtils.equalTuples(tokenTuple, getTokenTuple(tokenElement), tokenCmp.getKeyFieldCount())) {
+ initPushIntoKeyQueue(tokenElement);
+ tokenQueue.poll();
+ } else {
+ break;
+ }
+ }
+ }
+
+ private ITupleReference getKeyTuple(PriorityQueueElement tokenElement) {
+ return ((TokenKeyPairTuple) tokenElement.getTuple()).getKeyTuple();
+ }
+
+ private ITupleReference getTokenTuple(PriorityQueueElement tokenElement) {
+ return ((TokenKeyPairTuple) tokenElement.getTuple()).getTokenTuple();
+ }
+
+ private void initPushIntoKeyQueue(PriorityQueueElement tokenElement) {
+ PriorityQueueElement keyElement = keyQueueElements[tokenElement.getCursorIndex()];
+ keyElement.reset(getKeyTuple(tokenElement));
+ keyQueue.add(keyElement);
+ }
+
+ private void pushIntoKeyQueueAndReplace(PriorityQueueElement keyElement) throws HyracksDataException {
+ int cursorIndex = keyElement.getCursorIndex();
+ if (rangeCursors[cursorIndex].hasNext()) {
+ rangeCursors[cursorIndex].next();
+ TokenKeyPairTuple tuple = (TokenKeyPairTuple) rangeCursors[cursorIndex].getTuple();
+ if (tuple.isNewToken()) {
+ // if this element is a new token, then the current inverted list has exuasted
+ PriorityQueueElement tokenElement = tokenQueueElements[cursorIndex];
+ tokenElement.reset(tuple);
+ tokenQueue.offer(tokenElement);
+ } else {
+ keyElement.reset(tuple.getKeyTuple());
+ keyQueue.offer(keyElement);
+ }
+ } else {
+ rangeCursors[cursorIndex].close();
+ }
+ }
+
+ @Override
+ public boolean doHasNext() throws HyracksDataException {
+ checkPriorityQueue();
+ return !keyQueue.isEmpty();
+ }
+
+ @Override
+ public void doNext() throws HyracksDataException {
+ outputKeyElement = keyQueue.poll();
+ outputTuple.setKeyTuple(outputKeyElement.getTuple());
+ needPushElementIntoKeyQueue = true;
+ }
+
+ @Override
+ public ITupleReference doGetTuple() {
+ return outputTuple;
+ }
+
+ protected void checkPriorityQueue() throws HyracksDataException {
+ checkKeyQueue();
+ if (keyQueue.isEmpty()) {
+ // if key queue is empty, we search the next token and check again
+ searchNextToken();
+ checkKeyQueue();
+ }
+ }
+
+ protected void checkKeyQueue() throws HyracksDataException {
+ while (!keyQueue.isEmpty() || needPushElementIntoKeyQueue) {
+ if (!keyQueue.isEmpty()) {
+ PriorityQueueElement checkElement = keyQueue.peek();
+ // If there is no previous tuple or the previous tuple can be ignored
+ if (outputKeyElement == null) {
+ if (isDeleted(checkElement)) {
+ // If the key has been deleted then pop it and set needPush to true.
+ // We cannot push immediately because the tuple may be
+ // modified if hasNext() is called
+ outputKeyElement = checkElement;
+ needPushElementIntoKeyQueue = true;
+ } else {
+ // we have found the next record
+ return;
+ }
+ } else {
+ // Compare the previous tuple and the head tuple in the PQ
+ if (keyCmp.compare(outputKeyElement.getTuple(), checkElement.getTuple()) == 0) {
+ // If the previous tuple and the head tuple are
+ // identical
+ // then pop the head tuple and push the next tuple from
+ // the tree of head tuple
+
+ // the head element of PQ is useless now
+ PriorityQueueElement e = keyQueue.poll();
+ pushIntoKeyQueueAndReplace(e);
+ } else {
+ // If the previous tuple and the head tuple are different
+ // the info of previous tuple is useless
+ if (needPushElementIntoKeyQueue) {
+ pushIntoKeyQueueAndReplace(outputKeyElement);
+ needPushElementIntoKeyQueue = false;
+ }
+ outputKeyElement = null;
+ }
+ }
+ } else {
+ // the priority queue is empty and needPush
+ // NOSONAR: outputKeyElement is not null when needPushElementIntoKeyQueue = true
+ pushIntoKeyQueueAndReplace(outputKeyElement);
+ needPushElementIntoKeyQueue = false;
+ outputKeyElement = null;
+ }
+ }
+ }
+
+ /**
+ * Check deleted-keys BTrees whether they contain the key in the checkElement's tuple.
+ */
+ protected boolean isDeleted(PriorityQueueElement keyElement) throws HyracksDataException {
+ ITupleReference keyTuple = keyElement.getTuple();
+ int end = keyElement.getCursorIndex();
+ for (int i = 0; i < end; i++) {
+ if (bloomFilters[i] != null && !bloomFilters[i].contains(keyTuple, hashes)) {
+ continue;
+ }
+ deletedKeysBTreeCursors[i].close();
+ deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], deletedKeyBTreeSearchPred);
+ try {
+ if (deletedKeysBTreeCursors[i].hasNext()) {
+ return true;
+ }
+ } finally {
+ deletedKeysBTreeCursors[i].close();
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public void doClose() throws HyracksDataException {
+ outputTokenElement = null;
+ outputKeyElement = null;
+ needPushElementIntoKeyQueue = false;
+ try {
+ if (rangeCursors != null) {
+ for (int i = 0; i < rangeCursors.length; i++) {
+ rangeCursors[i].close();
+ }
+ }
+ } finally {
+ if (lsmHarness != null) {
+ lsmHarness.endSearch(opCtx);
+ }
+ }
+ }
+
+ @Override
+ public void doDestroy() throws HyracksDataException {
+ try {
+ if (tokenQueue != null) {
+ tokenQueue.clear();
+ }
+ if (keyQueue != null) {
+ keyQueue.clear();
+ }
+ if (rangeCursors != null) {
+ for (int i = 0; i < rangeCursors.length; i++) {
+ if (rangeCursors[i] != null) {
+ rangeCursors[i].destroy();
+ }
+ }
+ rangeCursors = null;
+ }
+ } finally {
+ if (lsmHarness != null) {
+ lsmHarness.endSearch(opCtx);
+ }
+ }
+ }
+
+ @Override
+ public ITupleReference getFilterMinTuple() {
+ return null;
+ }
+
+ @Override
+ public ITupleReference getFilterMaxTuple() {
+ return null;
+ }
+
+ @Override
+ public boolean getSearchOperationCallbackProceedResult() {
+ return false;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
index 12dc23f..020b788 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java
@@ -98,10 +98,9 @@
keysOnlyTuple.reset(checkElement.getTuple());
int end = checkElement.getCursorIndex();
for (int i = 0; i < end; i++) {
- if (bloomFilters[i] != null && bloomFilters[i].contains(keysOnlyTuple, hashes)) {
+ if (bloomFilters[i] != null && !bloomFilters[i].contains(keysOnlyTuple, hashes)) {
continue;
}
- deletedKeysBTreeCursors[i].close();
deletedKeysBTreeAccessors.get(i).search(deletedKeysBTreeCursors[i], keySearchPred);
try {
if (deletedKeysBTreeCursors[i].hasNext()) {
@@ -114,4 +113,31 @@
return false;
}
+ @Override
+ public void doClose() throws HyracksDataException {
+ try {
+ super.doClose();
+ } finally {
+ if (deletedKeysBTreeCursors != null) {
+ for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+ deletedKeysBTreeCursors[i].close();
+ }
+ }
+ }
+ }
+
+ @Override
+ public void doDestroy() throws HyracksDataException {
+ try {
+ super.doDestroy();
+ } finally {
+ if (deletedKeysBTreeCursors != null) {
+ for (int i = 0; i < deletedKeysBTreeCursors.length; i++) {
+ deletedKeysBTreeCursors[i].destroy();
+ }
+ deletedKeysBTreeCursors = null;
+ }
+ }
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
index 4d444b9..d39c601 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java
@@ -59,6 +59,7 @@
private List<IIndexAccessor> deletedKeysBTreeAccessors;
private RangePredicate keySearchPred;
private ILSMIndexOperationContext opCtx;
+ private boolean includeMemoryComponents;
private List<ILSMComponent> operationalComponents;
private ITupleReference currentTuple = null;
@@ -76,7 +77,7 @@
accessorIndex = 0;
this.searchPred = searchPred;
this.searchCallback = lsmInitState.getSearchOperationCallback();
-
+ includeMemoryComponents = false;
// For searching the deleted-keys BTrees.
deletedKeysBTreeAccessors = lsmInitState.getDeletedKeysBTreeAccessors();
deletedKeysBTreeCursors = new IIndexCursor[deletedKeysBTreeAccessors.size()];
@@ -87,6 +88,7 @@
if (component.getType() == LSMComponentType.MEMORY) {
// No need for a bloom filter for the in-memory BTree.
deletedKeysBTreeBloomFilters[i] = null;
+ includeMemoryComponents = true;
} else {
deletedKeysBTreeBloomFilters[i] = ((LSMInvertedIndexDiskComponent) component).getBloomFilter();
}
@@ -121,7 +123,8 @@
while (currentCursor.hasNext()) {
currentCursor.next();
currentTuple = currentCursor.getTuple();
- resultOfSearchCallBackProceed = searchCallback.proceed(currentTuple);
+ resultOfSearchCallBackProceed =
+ includeMemoryComponents && accessorIndex == 0 ? searchCallback.proceed(currentTuple) : true;
if (!resultOfSearchCallBackProceed) {
// We assume that the underlying cursors materialize their results such that
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 d7482a8..604d374 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
@@ -40,6 +40,7 @@
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.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
public class PartitionedLSMInvertedIndex extends LSMInvertedIndex {
@@ -52,12 +53,12 @@
IBinaryTokenizerFactory tokenizerFactory, ILSMMergePolicy mergePolicy, ILSMOperationTracker opTracker,
ILSMIOOperationScheduler ioScheduler, ILSMIOOperationCallbackFactory ioOpCallbackFactory,
int[] invertedIndexFields, int[] filterFields, int[] filterFieldsForNonBulkLoadOps,
- int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable) throws HyracksDataException {
+ 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, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable);
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
index da3f079..7f3d12f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListCursor.java
@@ -220,9 +220,11 @@
// Assumption: processing inverted list takes time; so, we don't want to keep them on the buffer cache.
// Rather, we utilize the assigned working memory (buffers).
tmpBuffer = page.getBuffer();
- tmpBuffer.rewind();
- buffers.get(currentBufferIdx).rewind();
- buffers.get(currentBufferIdx).put(tmpBuffer);
+
+ // Copies the entire content of the page to the current buffer in the working memory.
+ System.arraycopy(tmpBuffer.array(), 0, buffers.get(currentBufferIdx).array(), 0,
+ buffers.get(currentBufferIdx).capacity());
+ buffers.get(currentBufferIdx).position(buffers.get(currentBufferIdx).capacity());
currentBufferIdx++;
bufferCache.unpin(page);
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
index b033b99..2401c67 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/FixedSizeElementInvertedListScanCursor.java
@@ -53,6 +53,7 @@
protected ICachedPage page;
protected boolean pinned;
+ protected int pinnedPageId = -1;
public FixedSizeElementInvertedListScanCursor(IBufferCache bufferCache, int fileId, ITypeTraits[] invListFields)
throws HyracksDataException {
@@ -103,14 +104,18 @@
*/
@Override
public void loadPages() throws HyracksDataException {
- if (pinned) {
- unloadPages();
- }
if (currentPageId == endPageId) {
+ // inverted list exhausted, return
return;
}
currentPageId++;
+ if (pinned && pinnedPageId == currentPageId) {
+ // already pinned, return
+ return;
+ }
+ unloadPages();
page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), false);
+ pinnedPageId = currentPageId;
pinned = true;
}
@@ -134,7 +139,6 @@
// Deducts 1 since the startPage would be set to bufferCurrentPageId + 1 in loadPages().
this.currentPageId = startPageId - 1;
this.numPages = endPageId - startPageId + 1;
- this.pinned = false;
}
@Override
@@ -158,16 +162,14 @@
@Override
public void doClose() throws HyracksDataException {
- if (pinned) {
- unloadPages();
- }
+ // No op
+ // We allow the inverted list cursor to hold at most one page to avoid
+ // unnecessary pins
}
@Override
public void doDestroy() throws HyracksDataException {
- if (pinned) {
- unloadPages();
- }
+ unloadPages();
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
index 2f4f1d6..c3c9c21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java
@@ -51,6 +51,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexSearchCursorInitialState;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.TOccurrenceSearcher;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexAccessor;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -204,9 +205,9 @@
try {
if (ctx.getBtreeCursor().hasNext()) {
ctx.getBtreeCursor().next();
- openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor);
+ openInvertedListCursor(ctx.getBtreeCursor().getTuple(), listCursor, ctx);
} else {
- LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+ LSMInvertedIndexSearchCursorInitialState initState = ctx.getCursorInitialState();
initState.setInvertedListInfo(0, 0, 0, 0);
listCursor.open(initState, null);
}
@@ -215,8 +216,8 @@
}
}
- public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor)
- throws HyracksDataException {
+ public void openInvertedListCursor(ITupleReference btreeTuple, InvertedListCursor listCursor,
+ OnDiskInvertedIndexOpContext opCtx) throws HyracksDataException {
int startPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListStartPageIdField),
btreeTuple.getFieldStart(invListStartPageIdField));
int endPageId = IntegerPointable.getInteger(btreeTuple.getFieldData(invListEndPageIdField),
@@ -225,35 +226,33 @@
btreeTuple.getFieldStart(invListStartOffField));
int numElements = IntegerPointable.getInteger(btreeTuple.getFieldData(invListNumElementsField),
btreeTuple.getFieldStart(invListNumElementsField));
- LSMInvertedIndexSearchCursorInitialState initState = new LSMInvertedIndexSearchCursorInitialState();
+ LSMInvertedIndexSearchCursorInitialState initState = opCtx.getCursorInitialState();
initState.setInvertedListInfo(startPageId, endPageId, startOff, numElements);
listCursor.open(initState, null);
}
- public final class OnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
- private final ArrayTupleBuilder btreeTupleBuilder;
- private final ArrayTupleReference btreeTupleReference;
- private final IIndexBulkLoader btreeBulkloader;
+ public abstract class AbstractOnDiskInvertedIndexBulkLoader implements IIndexBulkLoader {
+ protected final ArrayTupleBuilder btreeTupleBuilder;
+ protected final ArrayTupleReference btreeTupleReference;
+ protected final IIndexBulkLoader btreeBulkloader;
- private int currentInvListStartPageId;
- private int currentInvListStartOffset;
- private final ArrayTupleBuilder lastTupleBuilder;
- private final ArrayTupleReference lastTuple;
+ protected int currentInvListStartPageId;
+ protected int currentInvListStartOffset;
+ protected final ArrayTupleBuilder lastTupleBuilder;
+ protected final ArrayTupleReference lastTuple;
- private int currentPageId;
- private ICachedPage currentPage;
- private final MultiComparator tokenCmp;
- private final MultiComparator invListCmp;
+ protected int currentPageId;
+ protected ICachedPage currentPage;
+ protected final MultiComparator invListCmp;
- private final boolean verifyInput;
- private final MultiComparator allCmp;
+ protected final boolean verifyInput;
+ protected final MultiComparator allCmp;
- private final IFIFOPageQueue queue;
+ protected final IFIFOPageQueue queue;
- public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+ public AbstractOnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
this.verifyInput = verifyInput;
- this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
this.invListCmp = MultiComparator.create(invListCmpFactories);
if (verifyInput) {
allCmp = MultiComparator.create(btree.getComparatorFactories(), invListCmpFactories);
@@ -272,22 +271,15 @@
queue = bufferCache.createFIFOQueue();
}
- public void pinNextPage() throws HyracksDataException {
+ protected void pinNextPage() throws HyracksDataException {
queue.put(currentPage);
currentPageId++;
currentPage = bufferCache.confiscatePage(BufferedFileHandle.getDiskPageId(fileId, currentPageId));
}
- private void createAndInsertBTreeTuple() throws HyracksDataException {
+ protected void insertBTreeTuple() throws HyracksDataException {
// Build tuple.
- btreeTupleBuilder.reset();
DataOutput output = btreeTupleBuilder.getDataOutput();
- // Add key fields.
- lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
- for (int i = 0; i < numTokenFields; i++) {
- btreeTupleBuilder.addField(lastTuple.getFieldData(i), lastTuple.getFieldStart(i),
- lastTuple.getFieldLength(i));
- }
// Add inverted-list 'pointer' value fields.
try {
output.writeInt(currentInvListStartPageId);
@@ -304,77 +296,59 @@
// Reset tuple reference and add it into the BTree load.
btreeTupleReference.reset(btreeTupleBuilder.getFieldEndOffsets(), btreeTupleBuilder.getByteArray());
btreeBulkloader.add(btreeTupleReference);
+ btreeTupleBuilder.reset();
}
- /**
- * Assumptions:
- * The first btree.getMultiComparator().getKeyFieldCount() fields in tuple
- * are btree keys (e.g., a string token).
- * The next invListCmp.getKeyFieldCount() fields in tuple are keys of the
- * inverted list (e.g., primary key).
- * Key fields of inverted list are fixed size.
- */
- @Override
- public void add(ITupleReference tuple) throws HyracksDataException {
- boolean firstElement = lastTupleBuilder.getSize() == 0;
- boolean startNewList = firstElement;
- if (!firstElement) {
- // If the current and the last token don't match, we start a new list.
- lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
- startNewList = tokenCmp.compare(tuple, lastTuple) != 0;
- }
- if (startNewList) {
- if (!firstElement) {
- // Create entry in btree for last inverted list.
- createAndInsertBTreeTuple();
- }
- if (!invListBuilder.startNewList(tuple, numTokenFields)) {
- pinNextPage();
- invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
- if (!invListBuilder.startNewList(tuple, numTokenFields)) {
- throw new IllegalStateException("Failed to create first inverted list.");
- }
- }
- currentInvListStartPageId = currentPageId;
- currentInvListStartOffset = invListBuilder.getPos();
- } else {
- if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
- // Duplicate inverted-list element.
- return;
- }
- }
-
- // Append to current inverted list.
- if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+ protected void startNewList(ITupleReference tokenTuple) throws HyracksDataException {
+ if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
pinNextPage();
invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
- if (!invListBuilder.appendElement(tuple, numTokenFields, numInvListKeys)) {
+ if (!invListBuilder.startNewList(tokenTuple, numTokenFields)) {
+ throw new IllegalStateException("Failed to create first inverted list.");
+ }
+ }
+ currentInvListStartPageId = currentPageId;
+ currentInvListStartOffset = invListBuilder.getPos();
+ }
+
+ protected void appendInvertedList(ITupleReference keyTuple, int startField) throws HyracksDataException {
+ if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
+ pinNextPage();
+ invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
+ if (!invListBuilder.appendElement(keyTuple, startField, numInvListKeys)) {
throw new IllegalStateException(
"Failed to append element to inverted list after switching to a new page.");
}
}
+ }
- if (verifyInput && lastTupleBuilder.getSize() != 0) {
- if (allCmp.compare(tuple, lastTuple) <= 0) {
- throw new HyracksDataException(
- "Input stream given to OnDiskInvertedIndex bulk load is not sorted.");
- }
+ protected void verifyTuple(ITupleReference tuple) throws HyracksDataException {
+ if (lastTupleBuilder.getSize() > 0 && allCmp.compare(tuple, lastTuple) <= 0) {
+ HyracksDataException.create(ErrorCode.UNSORTED_LOAD_INPUT);
}
+ }
- // Remember last tuple by creating a copy.
- // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
+ protected void saveLastTuple(ITupleReference tuple) throws HyracksDataException {
lastTupleBuilder.reset();
for (int i = 0; i < tuple.getFieldCount(); i++) {
lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
}
+ lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
+ }
+
+ protected void copyTokenToBTreeTuple(ITupleReference tokenTuple) throws HyracksDataException {
+ for (int i = 0; i < numTokenFields; i++) {
+ btreeTupleBuilder.addField(tokenTuple.getFieldData(i), tokenTuple.getFieldStart(i),
+ tokenTuple.getFieldLength(i));
+ }
}
@Override
public void end() throws HyracksDataException {
- // The last tuple builder is empty if add() was never called.
- if (lastTupleBuilder.getSize() != 0) {
- createAndInsertBTreeTuple();
+ if (btreeTupleBuilder.getSize() != 0) {
+ insertBTreeTuple();
}
+
btreeBulkloader.end();
if (currentPage != null) {
@@ -392,6 +366,72 @@
}
}
+ public class OnDiskInvertedIndexMergeBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+ public OnDiskInvertedIndexMergeBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+ super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ TokenKeyPairTuple pairTuple = (TokenKeyPairTuple) tuple;
+ ITupleReference tokenTuple = pairTuple.getTokenTuple();
+ ITupleReference keyTuple = pairTuple.getKeyTuple();
+ boolean startNewList = pairTuple.isNewToken();
+ if (startNewList) {
+ if (btreeTupleBuilder.getSize() > 0) {
+ insertBTreeTuple();
+ }
+ startNewList(tokenTuple);
+ copyTokenToBTreeTuple(tokenTuple);
+ }
+ appendInvertedList(keyTuple, 0);
+ if (verifyInput) {
+ verifyTuple(tuple);
+ saveLastTuple(tuple);
+ }
+ }
+ }
+
+ public class OnDiskInvertedIndexBulkLoader extends AbstractOnDiskInvertedIndexBulkLoader {
+
+ public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex, int startPageId) throws HyracksDataException {
+ super(btreeFillFactor, verifyInput, numElementsHint, checkIfEmptyIndex, startPageId);
+ }
+
+ @Override
+ public void add(ITupleReference tuple) throws HyracksDataException {
+ boolean firstElement = btreeTupleBuilder.getSize() == 0;
+ boolean startNewList = firstElement;
+ if (!firstElement) {
+ // If the current and the last token don't match, we start a new list.
+ startNewList = !TupleUtils.equalTuples(tuple, lastTuple, numTokenFields);
+ }
+ if (startNewList) {
+ if (!firstElement) {
+ // Create entry in btree for last inverted list.
+ insertBTreeTuple();
+ }
+ startNewList(tuple);
+ copyTokenToBTreeTuple(tuple);
+ } else {
+ if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
+ // Duplicate inverted-list element.
+ return;
+ }
+ }
+ appendInvertedList(tuple, numTokenFields);
+ if (verifyInput) {
+ verifyTuple(tuple);
+ }
+
+ saveLastTuple(tuple);
+ }
+
+ }
+
@Override
public IBufferCache getBufferCache() {
return bufferCache;
@@ -518,6 +558,12 @@
rootPageId);
}
+ public IIndexBulkLoader createMergeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
+ boolean checkIfEmptyIndex) throws HyracksDataException {
+ return new OnDiskInvertedIndexMergeBulkLoader(fillFactor, verifyInput, numElementsHint, checkIfEmptyIndex,
+ rootPageId);
+ }
+
@Override
public void validate() throws HyracksDataException {
btree.validate();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
index 267cc79..81bc46b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java
@@ -25,6 +25,7 @@
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.impls.LSMInvertedIndexSearchCursorInitialState;
import org.apache.hyracks.storage.common.IIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.apache.hyracks.storage.common.MultiComparator;
@@ -32,12 +33,13 @@
public class OnDiskInvertedIndexOpContext implements IIndexOperationContext {
private final RangePredicate btreePred = new RangePredicate(null, null, true, true, null, null);
- private IIndexAccessor btreeAccessor;
- private IIndexCursor btreeCursor;
- private MultiComparator searchCmp;
+ private final IIndexAccessor btreeAccessor;
+ private final IIndexCursor btreeCursor;
+ private final MultiComparator searchCmp;
// For prefix search on partitioned indexes.
private MultiComparator prefixSearchCmp;
private boolean destroyed = false;
+ private LSMInvertedIndexSearchCursorInitialState cursorInitialState;
public OnDiskInvertedIndexOpContext(BTree btree) throws HyracksDataException {
// TODO: Ignore opcallbacks for now.
@@ -96,4 +98,11 @@
btreeCursor.destroy();
}
}
+
+ public LSMInvertedIndexSearchCursorInitialState getCursorInitialState() {
+ if (cursorInitialState == null) {
+ cursorInitialState = new LSMInvertedIndexSearchCursorInitialState();
+ }
+ return cursorInitialState;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
index d9e7d34..11b483e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java
@@ -24,10 +24,9 @@
import org.apache.hyracks.storage.am.btree.impls.RangePredicate;
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.tuples.ConcatenatingTupleReference;
import org.apache.hyracks.storage.am.common.tuples.PermutingTupleReference;
-import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInPlaceInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
+import org.apache.hyracks.storage.am.lsm.invertedindex.tuples.TokenKeyPairTuple;
import org.apache.hyracks.storage.common.EnforcedIndexCursor;
import org.apache.hyracks.storage.common.ICursorInitialState;
import org.apache.hyracks.storage.common.IIndexAccessor;
@@ -41,7 +40,7 @@
private final BTree btree;
private final IIndexAccessor btreeAccessor;
- private final IInPlaceInvertedIndex invIndex;
+ private final OnDiskInvertedIndex invIndex;
private final IIndexOperationContext opCtx;
private final InvertedListCursor invListRangeSearchCursor;
private boolean isInvListCursorOpen;
@@ -50,11 +49,11 @@
private RangePredicate btreePred;
private final PermutingTupleReference tokenTuple;
- private ConcatenatingTupleReference concatTuple;
+ private final TokenKeyPairTuple resultTuple;
- public OnDiskInvertedIndexRangeSearchCursor(IInPlaceInvertedIndex invIndex, IIndexOperationContext opCtx)
+ public OnDiskInvertedIndexRangeSearchCursor(OnDiskInvertedIndex invIndex, IIndexOperationContext opCtx)
throws HyracksDataException {
- this.btree = ((OnDiskInvertedIndex) invIndex).getBTree();
+ this.btree = invIndex.getBTree();
this.btreeAccessor = btree.createAccessor(NoOpIndexAccessParameters.INSTANCE);
this.invIndex = invIndex;
this.opCtx = opCtx;
@@ -65,7 +64,7 @@
}
tokenTuple = new PermutingTupleReference(fieldPermutation);
btreeCursor = btreeAccessor.createSearchCursor(false);
- concatTuple = new ConcatenatingTupleReference(2);
+ resultTuple = new TokenKeyPairTuple(invIndex.getTokenTypeTraits().length, btree.getCmpFactories().length);
invListRangeSearchCursor = invIndex.createInvertedListRangeSearchCursor();
isInvListCursorOpen = false;
}
@@ -87,11 +86,7 @@
return true;
}
// The current inverted-list-range-search cursor is exhausted.
- try {
- invListRangeSearchCursor.unloadPages();
- } finally {
- invListRangeSearchCursor.close();
- }
+ invListRangeSearchCursor.close();
isInvListCursorOpen = false;
openInvListRangeSearchCursor();
return isInvListCursorOpen;
@@ -100,23 +95,14 @@
@Override
public void doNext() throws HyracksDataException {
invListRangeSearchCursor.next();
- if (concatTuple.hasMaxTuples()) {
- concatTuple.removeLastTuple();
- }
- concatTuple.addTuple(invListRangeSearchCursor.getTuple());
+ resultTuple.setKeyTuple(invListRangeSearchCursor.getTuple());
}
@Override
public void doDestroy() throws HyracksDataException {
try {
- if (isInvListCursorOpen) {
- try {
- invListRangeSearchCursor.unloadPages();
- } finally {
- isInvListCursorOpen = false;
- invListRangeSearchCursor.destroy();
- }
- }
+ invListRangeSearchCursor.destroy();
+ isInvListCursorOpen = false;
} finally {
btreeCursor.destroy();
}
@@ -125,14 +111,8 @@
@Override
public void doClose() throws HyracksDataException {
try {
- if (isInvListCursorOpen) {
- try {
- invListRangeSearchCursor.unloadPages();
- } finally {
- invListRangeSearchCursor.close();
- }
- isInvListCursorOpen = false;
- }
+ invListRangeSearchCursor.close();
+ isInvListCursorOpen = false;
} finally {
btreeCursor.close();
}
@@ -140,7 +120,7 @@
@Override
public ITupleReference doGetTuple() {
- return concatTuple;
+ return resultTuple;
}
// Opens an inverted-list-scan cursor for the given tuple.
@@ -148,11 +128,11 @@
if (btreeCursor.hasNext()) {
btreeCursor.next();
tokenTuple.reset(btreeCursor.getTuple());
- invIndex.openInvertedListCursor(invListRangeSearchCursor, tokenTuple, opCtx);
+ invIndex.openInvertedListCursor(btreeCursor.getTuple(), invListRangeSearchCursor,
+ (OnDiskInvertedIndexOpContext) opCtx);
invListRangeSearchCursor.prepareLoadPages();
invListRangeSearchCursor.loadPages();
- concatTuple.reset();
- concatTuple.addTuple(tokenTuple);
+ resultTuple.setTokenTuple(tokenTuple);
isInvListCursorOpen = true;
} else {
isInvListCursorOpen = false;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
index eff4f5a..8c6b386 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/ondisk/PartitionedOnDiskInvertedIndex.java
@@ -117,7 +117,7 @@
short numTokens = ShortPointable.getShort(btreeTuple.getFieldData(PARTITIONING_NUM_TOKENS_FIELD),
btreeTuple.getFieldStart(PARTITIONING_NUM_TOKENS_FIELD));
InvertedListCursor invListCursor = partSearcher.getCachedInvertedListCursor();
- openInvertedListCursor(btreeTuple, invListCursor);
+ openInvertedListCursor(btreeTuple, invListCursor, ctx);
invListPartitions.addInvertedListCursor(invListCursor, numTokens);
tokenExists = true;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
new file mode 100644
index 0000000..102fe96
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/tuples/TokenKeyPairTuple.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.tuples;
+
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class TokenKeyPairTuple implements ITupleReference {
+
+ private ITupleReference tokenTuple;
+ private ITupleReference keyTuple;
+
+ private final int tokenFieldCount;
+ private final int keyFieldCount;
+
+ private boolean newToken;
+
+ public TokenKeyPairTuple(int tokenFieldCount, int keyFieldCount) {
+ this.tokenFieldCount = tokenFieldCount;
+ this.keyFieldCount = keyFieldCount;
+
+ }
+
+ public void setTokenTuple(ITupleReference token) {
+ this.tokenTuple = token;
+ this.keyTuple = null;
+ }
+
+ public void setKeyTuple(ITupleReference key) {
+ newToken = this.keyTuple == null;
+ this.keyTuple = key;
+ }
+
+ public ITupleReference getTokenTuple() {
+ return tokenTuple;
+ }
+
+ public ITupleReference getKeyTuple() {
+ return keyTuple;
+ }
+
+ @Override
+ public int getFieldCount() {
+ return tokenFieldCount + keyFieldCount;
+ }
+
+ @Override
+ public byte[] getFieldData(int fIdx) {
+ ITupleReference tuple = getTuple(fIdx);
+ int fieldIndex = getFieldIndex(fIdx);
+ return tuple.getFieldData(fieldIndex);
+ }
+
+ @Override
+ public int getFieldStart(int fIdx) {
+ ITupleReference tuple = getTuple(fIdx);
+ int fieldIndex = getFieldIndex(fIdx);
+ return tuple.getFieldStart(fieldIndex);
+ }
+
+ @Override
+ public int getFieldLength(int fIdx) {
+ ITupleReference tuple = getTuple(fIdx);
+ int fieldIndex = getFieldIndex(fIdx);
+ return tuple.getFieldLength(fieldIndex);
+ }
+
+ private ITupleReference getTuple(int fIdx) {
+ return fIdx < tokenFieldCount ? tokenTuple : keyTuple;
+ }
+
+ private int getFieldIndex(int fIdx) {
+ return fIdx < tokenFieldCount ? fIdx : fIdx - tokenFieldCount;
+ }
+
+ public boolean isNewToken() {
+ return newToken;
+ }
+}
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 ba54ed4..1e812b4 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
@@ -62,6 +62,7 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.ondisk.PartitionedOnDiskInvertedIndexFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
+import org.apache.hyracks.util.trace.ITracer;
public class InvertedIndexUtils {
@@ -129,7 +130,7 @@
ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- IMetadataPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ IMetadataPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -166,7 +167,7 @@
filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager, invListTypeTraits,
invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory, mergePolicy, opTracker,
ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, durable);
+ invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
public static PartitionedLSMInvertedIndex createPartitionedLSMInvertedIndex(IIOManager ioManager,
@@ -178,7 +179,7 @@
ILSMIOOperationCallbackFactory ioOpCallbackFactory, int[] invertedIndexFields,
ITypeTraits[] filterTypeTraits, IBinaryComparatorFactory[] filterCmpFactories, int[] filterFields,
int[] filterFieldsForNonBulkLoadOps, int[] invertedIndexFieldsForNonBulkLoadOps, boolean durable,
- IPageManagerFactory pageManagerFactory) throws HyracksDataException {
+ IPageManagerFactory pageManagerFactory, ITracer tracer) throws HyracksDataException {
BTreeFactory deletedKeysBTreeFactory = createDeletedKeysBTreeFactory(ioManager, invListTypeTraits,
invListCmpFactories, diskBufferCache, pageManagerFactory);
@@ -215,6 +216,6 @@
filterFrameFactory, filterManager, bloomFilterFalsePositiveRate, diskBufferCache, fileManager,
invListTypeTraits, invListCmpFactories, tokenTypeTraits, tokenCmpFactories, tokenizerFactory,
mergePolicy, opTracker, ioScheduler, ioOpCallbackFactory, invertedIndexFields, filterFields,
- filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable);
+ filterFieldsForNonBulkLoadOps, invertedIndexFieldsForNonBulkLoadOps, durable, tracer);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index e36abb4..c695a5c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -97,9 +97,8 @@
BTree memBTree = new BTree(virtualBufferCache, new VirtualFreePageManager(virtualBufferCache),
btreeInteriorFrameFactory, btreeLeafFrameFactory, btreeCmpFactories, btreeCmpFactories.length,
ioManager.resolveAbsolutePath(fileManager.getBaseDir() + "_virtual_b_" + i));
- LSMRTreeMemoryComponent mutableComponent =
- new LSMRTreeMemoryComponent(this, memRTree, memBTree, virtualBufferCache, i == 0 ? true : false,
- filterHelper == null ? null : filterHelper.createFilter());
+ LSMRTreeMemoryComponent mutableComponent = new LSMRTreeMemoryComponent(this, memRTree, memBTree,
+ virtualBufferCache, filterHelper == null ? null : filterHelper.createFilter());
memoryComponents.add(mutableComponent);
++i;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
index 7b12250..f902153 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/ExternalRTree.java
@@ -21,6 +21,7 @@
import java.io.IOException;
import java.util.LinkedList;
import java.util.List;
+import java.util.Map;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
@@ -30,7 +31,6 @@
import org.apache.hyracks.storage.am.bloomfilter.impls.BloomCalculations;
import org.apache.hyracks.storage.am.bloomfilter.impls.BloomFilterSpecification;
import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeBulkLoader;
import org.apache.hyracks.storage.am.common.api.IIndexOperationContext;
import org.apache.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import org.apache.hyracks.storage.am.common.api.ITwoPCIndexBulkLoader;
@@ -40,8 +40,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -50,9 +48,10 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import org.apache.hyracks.storage.am.lsm.common.api.ITwoPCIndex;
-import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.impls.ChainedLSMDiskComponentBulkLoader;
import org.apache.hyracks.storage.am.lsm.common.impls.ExternalIndexHarness;
import org.apache.hyracks.storage.am.lsm.common.impls.LSMComponentFileReferences;
+import org.apache.hyracks.storage.am.lsm.common.impls.LoadOperation;
import org.apache.hyracks.storage.am.rtree.impls.SearchPredicate;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
import org.apache.hyracks.storage.common.IIndexBulkLoader;
@@ -323,12 +322,6 @@
if (!isActive) {
throw new HyracksDataException("Failed to deactivate the index since it is already deactivated.");
}
- if (flushOnExit) {
- AbstractLSMIndexOperationContext opCtx = createOpContext(NoOpOperationCallback.INSTANCE, version);
- opCtx.setIoOperationType(LSMIOOperationType.FLUSH);
- ioOpCallback.afterFinalize(opCtx);
- }
-
for (ILSMDiskComponent c : diskComponents) {
c.deactivateAndPurge();
}
@@ -383,16 +376,13 @@
version = 0;
}
- // Not supported
@Override
public void modify(IIndexOperationContext ictx, ITupleReference tuple) throws HyracksDataException {
throw new UnsupportedOperationException("tuple modify not supported in LSM-Disk-Only-RTree");
}
- // Not supported
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
+ public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
throw new UnsupportedOperationException("flush not supported in LSM-Disk-Only-RTree");
}
@@ -445,139 +435,106 @@
// For initial load
@Override
- public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false);
+ public IIndexBulkLoader createBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, 0, false, parameters);
}
// For transaction bulk load <- could consolidate with the above method ->
@Override
- public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint)
- throws HyracksDataException {
- return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true);
+ public IIndexBulkLoader createTransactionBulkLoader(float fillLevel, boolean verifyInput, long numElementsHint,
+ Map<String, Object> parameters) throws HyracksDataException {
+ return new LSMTwoPCRTreeBulkLoader(fillLevel, verifyInput, numElementsHint, true, parameters);
}
// The bulk loader used for both initial loading and transaction
// modifications
public class LSMTwoPCRTreeBulkLoader implements IIndexBulkLoader, ITwoPCIndexBulkLoader {
private final ILSMDiskComponent component;
- private final IIndexBulkLoader rtreeBulkLoader;
- private final BTreeBulkLoader btreeBulkLoader;
- private final IIndexBulkLoader builder;
- private boolean cleanedUpArtifacts = false;
- private boolean isEmptyComponent = true;
- private boolean endedBloomFilterLoad = false;
private final boolean isTransaction;
+ private final LoadOperation loadOp;
+ private final ChainedLSMDiskComponentBulkLoader componentBulkLoader;
public LSMTwoPCRTreeBulkLoader(float fillFactor, boolean verifyInput, long numElementsHint,
- boolean isTransaction) throws HyracksDataException {
+ boolean isTransaction, Map<String, Object> parameters) throws HyracksDataException {
this.isTransaction = isTransaction;
// Create the appropriate target
+ LSMComponentFileReferences componentFileRefs;
if (isTransaction) {
- component = createTransactionTarget();
+ try {
+ componentFileRefs = fileManager.getNewTransactionFileReference();
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ component = createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
+ componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true);
} else {
- component = createBulkLoadTarget();
+ componentFileRefs = fileManager.getRelFlushFileReference();
+ component =
+ createDiskComponent(bulkLoadComponentFactory, componentFileRefs.getInsertIndexFileReference(),
+ componentFileRefs.getDeleteIndexFileReference(),
+ componentFileRefs.getBloomFilterFileReference(), true);
}
- // Create the three loaders
- rtreeBulkLoader = ((LSMRTreeDiskComponent) component).getIndex().createBulkLoader(fillFactor, verifyInput,
- numElementsHint, false);
- btreeBulkLoader = (BTreeBulkLoader) ((LSMRTreeDiskComponent) component).getBuddyIndex()
- .createBulkLoader(fillFactor, verifyInput, numElementsHint, false);
- int maxBucketsPerElement = BloomCalculations.maxBucketsPerElement(numElementsHint);
- BloomFilterSpecification bloomFilterSpec =
- BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate);
- builder = ((LSMRTreeDiskComponent) component).getBloomFilter().createBuilder(numElementsHint,
- bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements());
+ loadOp = new LoadOperation(componentFileRefs, ioOpCallback, getIndexIdentifier(), parameters);
+ loadOp.setNewComponent(component);
+ ioOpCallback.scheduled(loadOp);
+ ioOpCallback.beforeOperation(loadOp);
+ componentBulkLoader =
+ component.createBulkLoader(loadOp, fillFactor, verifyInput, numElementsHint, false, true, false);
}
@Override
public void add(ITupleReference tuple) throws HyracksDataException {
- try {
- rtreeBulkLoader.add(tuple);
- } catch (Exception e) {
- cleanupArtifacts();
- throw e;
- }
- if (isEmptyComponent) {
- isEmptyComponent = false;
- }
- }
-
- // This is made public in case of a failure, it is better to delete all
- // created artifacts.
- public void cleanupArtifacts() throws HyracksDataException {
- if (!cleanedUpArtifacts) {
- cleanedUpArtifacts = true;
- ((LSMRTreeDiskComponent) component).deactivateAndDestroy();
- }
+ componentBulkLoader.add(tuple);
}
@Override
public void end() throws HyracksDataException {
- if (!cleanedUpArtifacts) {
- if (!endedBloomFilterLoad) {
- builder.end();
- endedBloomFilterLoad = true;
+ try {
+ ioOpCallback.afterOperation(loadOp);
+ componentBulkLoader.end();
+ if (component.getComponentSize() > 0) {
+ if (isTransaction) {
+ // Since this is a transaction component, validate and
+ // deactivate. it could later be added or deleted
+ component.markAsValid(durable);
+ ioOpCallback.afterFinalize(loadOp);
+ component.deactivate();
+ } else {
+ ioOpCallback.afterFinalize(loadOp);
+ getHarness().addBulkLoadedComponent(component);
+ }
}
- rtreeBulkLoader.end();
- btreeBulkLoader.end();
- if (isEmptyComponent) {
- cleanupArtifacts();
- } else if (isTransaction) {
- // Since this is a transaction component, validate and
- // deactivate. it could later be added or deleted
- component.markAsValid(durable);
- component.deactivate();
- } else {
- getHarness().addBulkLoadedComponent(component);
- }
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
@Override
public void delete(ITupleReference tuple) throws HyracksDataException {
- try {
- btreeBulkLoader.add(tuple);
- builder.add(tuple);
- } catch (Exception e) {
- cleanupArtifacts();
- throw e;
- }
- if (isEmptyComponent) {
- isEmptyComponent = false;
- }
+ componentBulkLoader.delete(tuple);
}
@Override
- public void abort() {
+ public void abort() throws HyracksDataException {
try {
- cleanupArtifacts();
- } catch (Exception e) {
-
+ try {
+ componentBulkLoader.abort();
+ } finally {
+ ioOpCallback.afterFinalize(loadOp);
+ }
+ } finally {
+ ioOpCallback.completed(loadOp);
}
}
-
- // This method is used to create a target for a bulk modify operation. This
- // component must then eventually be either committed or deleted
- private ILSMDiskComponent createTransactionTarget() throws HyracksDataException {
- LSMComponentFileReferences componentFileRefs;
- try {
- componentFileRefs = fileManager.getNewTransactionFileReference();
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
- return createDiskComponent(componentFactory, componentFileRefs.getInsertIndexFileReference(),
- componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
- true);
- }
}
// The only change the the schedule merge is the method used to create the
// opCtx. first line <- in schedule merge, we->
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
+ public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
ILSMIndexOperationContext rctx = createOpContext(NoOpOperationCallback.INSTANCE, -1);
rctx.setOperation(IndexOperation.MERGE);
List<ILSMComponent> mergingComponents = ctx.getComponentHolder();
@@ -587,10 +544,12 @@
(ILSMDiskComponent) mergingComponents.get(mergingComponents.size() - 1));
ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), rctx, buddyBTreeFields);
// create the merge operation.
- LSMRTreeMergeOperation mergeOp = new LSMRTreeMergeOperation(accessor, cursor,
- relMergeFileRefs.getInsertIndexFileReference(), relMergeFileRefs.getDeleteIndexFileReference(),
- relMergeFileRefs.getBloomFilterFileReference(), callback, fileManager.getBaseDir().getAbsolutePath());
- ioScheduler.scheduleOperation(mergeOp);
+ LSMRTreeMergeOperation mergeOp =
+ new LSMRTreeMergeOperation(accessor, cursor, relMergeFileRefs.getInsertIndexFileReference(),
+ relMergeFileRefs.getDeleteIndexFileReference(), relMergeFileRefs.getBloomFilterFileReference(),
+ ioOpCallback, fileManager.getBaseDir().getAbsolutePath());
+ ioOpCallback.scheduled(mergeOp);
+ return mergeOp;
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 4510618..4566d32 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -134,8 +134,8 @@
rTreeTupleSorter.sort();
component = createDiskComponent(componentFactory, flushOp.getTarget(), flushOp.getBTreeTarget(),
flushOp.getBloomFilterTarget(), true);
- componentBulkLoader =
- component.createBulkLoader(1.0f, false, numBTreeTuples.longValue(), false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, numBTreeTuples.longValue(),
+ false, false, false);
flushLoadRTree(isEmpty, rTreeTupleSorter, componentBulkLoader);
// scan the memory BTree and bulk load delete tuples
flushLoadBtree(memBTreeAccessor, componentBulkLoader, btreeNullPredicate);
@@ -332,11 +332,12 @@
.getNumElements();
}
componentBulkLoader =
- mergedComponent.createBulkLoader(1.0f, false, numElements, false, false, false);
+ mergedComponent.createBulkLoader(mergeOp, 1.0f, false, numElements, false, false, false);
mergeLoadBTree(opCtx, rtreeSearchPred, componentBulkLoader);
} else {
//no buddy-btree needed
- componentBulkLoader = mergedComponent.createBulkLoader(1.0f, false, 0L, false, false, false);
+ componentBulkLoader =
+ mergedComponent.createBulkLoader(mergeOp, 1.0f, false, 0L, false, false, false);
}
//search old rtree components
while (cursor.hasNext()) {
@@ -423,7 +424,7 @@
LSMRTreeAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(),
componentFileRefs.getDeleteIndexFileReference(), componentFileRefs.getBloomFilterFileReference(),
- callback, fileManager.getBaseDir().getAbsolutePath());
+ callback, getIndexIdentifier());
}
@Override
@@ -433,6 +434,6 @@
ILSMIndexAccessor accessor = new LSMRTreeAccessor(getHarness(), opCtx, buddyBTreeFields);
return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(),
mergeFileRefs.getDeleteIndexFileReference(), mergeFileRefs.getBloomFilterFileReference(), callback,
- fileManager.getBaseDir().getAbsolutePath());
+ getIndexIdentifier());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
index 2e1ba68..ef7b815 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeMemoryComponent.java
@@ -21,9 +21,9 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.btree.impls.BTree;
-import org.apache.hyracks.storage.am.lsm.common.api.AbstractLSMWithBuddyMemoryComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
+import org.apache.hyracks.storage.am.lsm.common.impls.AbstractLSMWithBuddyMemoryComponent;
import org.apache.hyracks.storage.am.rtree.impls.RTree;
public class LSMRTreeMemoryComponent extends AbstractLSMWithBuddyMemoryComponent {
@@ -32,8 +32,8 @@
private final BTree btree;
public LSMRTreeMemoryComponent(AbstractLSMRTree lsmIndex, RTree rtree, BTree btree, IVirtualBufferCache vbc,
- boolean isActive, ILSMComponentFilter filter) {
- super(lsmIndex, vbc, isActive, filter);
+ ILSMComponentFilter filter) {
+ super(lsmIndex, vbc, filter);
this.rtree = rtree;
this.btree = btree;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
index c79735f..b9e174f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeSearchCursor.java
@@ -98,7 +98,8 @@
rtreeCursors[currentCursor].next();
ITupleReference currentTuple = rtreeCursors[currentCursor].getTuple();
// Call proceed() to do necessary operations before returning this tuple.
- resultOfsearchCallbackProceed = searchCallback.proceed(currentTuple);
+ resultOfsearchCallbackProceed =
+ currentCursor == 0 && includeMutableComponent ? searchCallback.proceed(currentTuple) : true;
btreeTuple.reset(rtreeCursors[currentCursor].getTuple());
boolean killerTupleFound = false;
for (int i = 0; i < currentCursor && !killerTupleFound; i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
index a3ba4b1..2757595 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java
@@ -109,7 +109,7 @@
try {
memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate);
component = createDiskComponent(componentFactory, flushOp.getTarget(), null, null, true);
- componentBulkLoader = component.createBulkLoader(1.0f, false, 0L, false, false, false);
+ componentBulkLoader = component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
// Since the LSM-RTree is used as a secondary assumption, the
// primary key will be the last comparator in the BTree comparators
rTreeTupleSorter = new TreeTupleSorter(flushingComponent.getIndex().getFileId(), linearizerArray,
@@ -235,7 +235,7 @@
ILSMDiskComponent component = createDiskComponent(componentFactory, mergeOp.getTarget(), null, null, true);
ILSMDiskComponentBulkLoader componentBulkLoader =
- component.createBulkLoader(1.0f, false, 0L, false, false, false);
+ component.createBulkLoader(operation, 1.0f, false, 0L, false, false, false);
try {
while (cursor.hasNext()) {
cursor.next();
@@ -273,7 +273,7 @@
throws HyracksDataException {
ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
return new LSMRTreeFlushOperation(accessor, componentFileRefs.getInsertIndexFileReference(), null, null,
- callback, fileManager.getBaseDir().getAbsolutePath());
+ callback, getIndexIdentifier());
}
@Override
@@ -288,6 +288,6 @@
new LSMRTreeWithAntiMatterTuplesSearchCursor(opCtx, returnDeletedTuples);
ILSMIndexAccessor accessor = new LSMTreeIndexAccessor(getHarness(), opCtx, cursorFactory);
return new LSMRTreeMergeOperation(accessor, cursor, mergeFileRefs.getInsertIndexFileReference(), null, null,
- callback, fileManager.getBaseDir().getAbsolutePath());
+ callback, getIndexIdentifier());
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
index 7db65bd..05dc3ef 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesSearchCursor.java
@@ -177,7 +177,7 @@
// reconcile() and complete() can be added later after considering the semantics.
// Call proceed() to do necessary operations before returning this tuple.
- resultOfsearchCallBackProceed = searchCallback.proceed(diskRTreeTuple);
+ resultOfsearchCallBackProceed = true;
if (searchMemBTrees(diskRTreeTuple, numMemoryComponents)) {
// anti-matter tuple is NOT found
foundNext = true;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
deleted file mode 100644
index 1ca75cb..0000000
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/api/IGenericPrimitiveSerializerDeserializer.java
+++ /dev/null
@@ -1,26 +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.hyracks.storage.am.rtree.api;
-
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-
-public interface IGenericPrimitiveSerializerDeserializer<T> extends ISerializerDeserializer<T> {
- public double getValue(byte[] bytes, int offset);
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
index 18ced6d..4dcfa94 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorNodePushable.java
@@ -55,8 +55,9 @@
boolean appendIndexFilter, boolean appendOpCallbackProceedResult,
byte[] searchCallbackProceedResultFalseValue, byte[] searchCallbackProceedResultTrueValue)
throws HyracksDataException {
+ // TODO: predicate & limit pushdown not enabled for RTree yet
super(ctx, inputRecDesc, partition, minFilterFieldIndexes, maxFilterFieldIndexes, indexHelperFactory,
- retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter,
+ retainInput, retainMissing, missingWriterFactory, searchCallbackFactory, appendIndexFilter, null, -1,
appendOpCallbackProceedResult, searchCallbackProceedResultFalseValue,
searchCallbackProceedResultTrueValue);
if (keyFields != null && keyFields.length > 0) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
index fc54903..d6c954e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/IIndexBulkLoader.java
@@ -35,7 +35,7 @@
public void add(ITupleReference tuple) throws HyracksDataException;
/**
- * Finalize the bulk loading operation in the given context.
+ * Finalize the bulk loading operation in the given context and release all resources.
*
* @throws HyracksDataException
* If the BufferCache throws while un/pinning or un/latching.
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 1443bbc..55ed75e 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -77,7 +77,7 @@
private final Queue<BufferCacheHeaderHelper> headerPageCache = new ConcurrentLinkedQueue<>();
//DEBUG
- private Level fileOpsLevel = Level.DEBUG;
+ private static final Level fileOpsLevel = Level.DEBUG;
private ArrayList<CachedPage> confiscatedPages;
private Lock confiscateLock;
private HashMap<CachedPage, StackTraceElement[]> confiscatedPagesOwner;
@@ -607,33 +607,36 @@
void write(CachedPage cPage) throws HyracksDataException {
BufferedFileHandle fInfo = getFileInfo(cPage);
+ if (fInfo == null) {
+ throw new IllegalStateException("Attempting to write non-existing file");
+ }
// synchronize on fInfo to prevent the file handle from being deleted until the page is written.
synchronized (fInfo) {
- if (!fInfo.fileHasBeenDeleted()) {
- ByteBuffer buf = cPage.buffer.duplicate();
- final int totalPages = cPage.getFrameSizeMultiplier();
- final int extraBlockPageId = cPage.getExtraBlockPageId();
- final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
- BufferCacheHeaderHelper header = checkoutHeaderHelper();
- try {
- buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
- buf.position(0);
- long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
- getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)),
- header.prepareWrite(cPage, buf));
+ ByteBuffer buf = cPage.buffer.duplicate();
+ final int totalPages = cPage.getFrameSizeMultiplier();
+ final int extraBlockPageId = cPage.getExtraBlockPageId();
+ final boolean contiguousLargePages = (BufferedFileHandle.getPageId(cPage.dpid) + 1) == extraBlockPageId;
+ BufferCacheHeaderHelper header = checkoutHeaderHelper();
+ try {
+ buf.limit(contiguousLargePages ? pageSize * totalPages : pageSize);
+ buf.position(0);
+ long bytesWritten = ioManager.syncWrite(fInfo.getFileHandle(),
+ getOffsetForPage(BufferedFileHandle.getPageId(cPage.dpid)), header.prepareWrite(cPage, buf));
- if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
- + getPageSizeWithHeader()) {
- throw new HyracksDataException("Failed to write completely: " + bytesWritten);
- }
- } finally {
- returnHeaderHelper(header);
+ if (bytesWritten != (contiguousLargePages ? pageSize * (totalPages - 1) : 0)
+ + getPageSizeWithHeader()) {
+ throw new HyracksDataException("Failed to write completely: " + bytesWritten);
}
- if (totalPages > 1 && !contiguousLargePages) {
- buf.limit(totalPages * pageSize);
- ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
- }
- assert buf.capacity() == (pageSize * totalPages);
+ } finally {
+ returnHeaderHelper(header);
+ }
+ if (totalPages > 1 && !contiguousLargePages) {
+ buf.limit(totalPages * pageSize);
+ ioManager.syncWrite(fInfo.getFileHandle(), getOffsetForPage(extraBlockPageId), buf);
+ }
+ if (buf.capacity() != pageSize * totalPages) {
+ throw new IllegalStateException("Illegal number of bytes written, expected bytes written: "
+ + pageSize * totalPages + " actual bytes writte: " + buf.capacity());
}
}
}
@@ -786,11 +789,8 @@
synchronized (fileInfoMap) {
fileInfoMap.forEach((key, value) -> {
try {
- boolean fileHasBeenDeleted = value.fileHasBeenDeleted();
- sweepAndFlush(key, !fileHasBeenDeleted);
- if (!fileHasBeenDeleted) {
- ioManager.close(value.getFileHandle());
- }
+ sweepAndFlush(key, true);
+ ioManager.close(value.getFileHandle());
} catch (HyracksDataException e) {
if (LOGGER.isWarnEnabled()) {
LOGGER.log(Level.WARN, "Error flushing file id: " + key, e);
@@ -827,15 +827,15 @@
if (LOGGER.isEnabled(fileOpsLevel)) {
LOGGER.log(fileOpsLevel, "Opening file: " + fileRef + " in cache: " + this);
}
- int fileId;
+ int fileId = -1;
synchronized (fileInfoMap) {
if (fileMapManager.isMapped(fileRef)) {
fileId = fileMapManager.lookupFileId(fileRef);
} else {
fileId = fileMapManager.registerFile(fileRef);
}
- openFile(fileId);
}
+ openFile(fileId);
return fileId;
}
@@ -844,42 +844,60 @@
if (LOGGER.isEnabled(fileOpsLevel)) {
LOGGER.log(fileOpsLevel, "Opening file: " + fileId + " in cache: " + this);
}
- synchronized (fileInfoMap) {
- BufferedFileHandle fInfo;
- fInfo = fileInfoMap.get(fileId);
- if (fInfo == null) {
- boolean unreferencedFileFound = true;
- while (fileInfoMap.size() >= maxOpenFiles && unreferencedFileFound) {
- // map is full, make room by cleaning up unreferenced files
- unreferencedFileFound = false;
- for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
- if (entry.getValue().getReferenceCount() <= 0) {
- int entryFileId = entry.getKey();
- boolean fileHasBeenDeleted = entry.getValue().fileHasBeenDeleted();
- sweepAndFlush(entryFileId, !fileHasBeenDeleted);
- if (!fileHasBeenDeleted) {
- ioManager.close(entry.getValue().getFileHandle());
- }
- fileInfoMap.remove(entryFileId);
- unreferencedFileFound = true;
- // for-each iterator is invalid because we changed
- // fileInfoMap
- break;
+ BufferedFileHandle fInfo = null;
+ try {
+ fInfo = getOrCreateFileHandle(fileId);
+ if (fInfo.getFileHandle() == null) {
+ // a new file
+ synchronized (fInfo) {
+ // prevent concurrent opening of the same file
+ if (fInfo.getFileHandle() == null) {
+ if (fileInfoMap.size() > maxOpenFiles) {
+ closeOpeningFiles(fInfo);
}
+ // create, open, and map new file reference
+ FileReference fileRef = fileMapManager.lookupFileName(fileId);
+ IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ fInfo.setFileHandle(fh);
}
}
- if (fileInfoMap.size() >= maxOpenFiles) {
- throw new HyracksDataException("Could not open fileId " + fileId + ". Max number of files "
- + maxOpenFiles + " already opened and referenced.");
- }
- // create, open, and map new file reference
- FileReference fileRef = fileMapManager.lookupFileName(fileId);
- IFileHandle fh = ioManager.open(fileRef, IIOManager.FileReadWriteMode.READ_WRITE,
- IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
- fInfo = new BufferedFileHandle(fileId, fh);
- fileInfoMap.put(fileId, fInfo);
}
fInfo.incReferenceCount();
+ } catch (Exception e) {
+ removeFileInfo(fileId);
+ throw HyracksDataException.create(e);
+ }
+ }
+
+ private void closeOpeningFiles(BufferedFileHandle newFileHandle) throws HyracksDataException {
+ synchronized (fileInfoMap) {
+ boolean unreferencedFileFound = true;
+ while (fileInfoMap.size() > maxOpenFiles && unreferencedFileFound) {
+ // map is full, make room by cleaning up unreferenced files
+ unreferencedFileFound = false;
+ for (Map.Entry<Integer, BufferedFileHandle> entry : fileInfoMap.entrySet()) {
+ BufferedFileHandle fh = entry.getValue();
+ if (fh != newFileHandle && fh.getReferenceCount() <= 0) {
+ if (fh.getReferenceCount() < 0) {
+ throw new IllegalStateException("Illegal reference count " + fh.getReferenceCount()
+ + " of file " + fh.getFileHandle().getFileReference());
+ }
+ int entryFileId = entry.getKey();
+ sweepAndFlush(entryFileId, true);
+ ioManager.close(entry.getValue().getFileHandle());
+ fileInfoMap.remove(entryFileId);
+ unreferencedFileFound = true;
+ // for-each iterator is invalid because we changed
+ // fileInfoMap
+ break;
+ }
+ }
+ }
+ if (fileInfoMap.size() > maxOpenFiles) {
+ throw new HyracksDataException("Could not open fileId " + newFileHandle.getFileId()
+ + ". Max number of files " + maxOpenFiles + " already opened and referenced.");
+ }
}
}
@@ -977,15 +995,19 @@
@Override
public void deleteFile(FileReference fileRef) throws HyracksDataException {
+ boolean mapped = false;
+ int fileId = -1;
synchronized (fileInfoMap) {
if (fileMapManager.isMapped(fileRef)) {
- int fileId = fileMapManager.lookupFileId(fileRef);
- deleteFile(fileId);
- return;
- } else {
- IoUtil.delete(fileRef);
+ mapped = true;
+ fileId = fileMapManager.lookupFileId(fileRef);
}
}
+ if (mapped) {
+ deleteFile(fileId);
+ } else {
+ IoUtil.delete(fileRef);
+ }
}
@Override
@@ -993,35 +1015,30 @@
if (LOGGER.isEnabled(fileOpsLevel)) {
LOGGER.log(fileOpsLevel, "Deleting file: " + fileId + " in cache: " + this);
}
- synchronized (fileInfoMap) {
- sweepAndFlush(fileId, false);
- BufferedFileHandle fInfo = null;
+ BufferedFileHandle fInfo = removeFileInfo(fileId);
+ if (fInfo == null) {
+ return;
+ }
+ sweepAndFlush(fileId, false);
+ try {
+ if (fInfo.getReferenceCount() > 0) {
+ throw new HyracksDataException("Deleting open file");
+ }
+ } finally {
+ FileReference fileRef = null;
try {
- fInfo = fileInfoMap.get(fileId);
- if (fInfo != null && fInfo.getReferenceCount() > 0) {
- throw new HyracksDataException("Deleting open file");
+ synchronized (fileInfoMap) {
+ fileRef = fileMapManager.unregisterFile(fileId);
}
- } catch (Exception e) {
- throw HyracksDataException.create(e);
} finally {
- FileReference fileRef = fileMapManager.unregisterFile(fileId);
try {
- if (fInfo != null) {
- // Mark the fInfo as deleted,
- // such that when its pages are reclaimed in openFile(),
- // the pages are not flushed to disk but only invalidated.
- synchronized (fInfo) {
- if (!fInfo.fileHasBeenDeleted()) {
- ioManager.close(fInfo.getFileHandle());
- fInfo.markAsDeleted();
- }
- }
- }
+ ioManager.close(fInfo.getFileHandle());
} finally {
IoUtil.delete(fileRef);
}
}
}
+
}
@Override
@@ -1264,6 +1281,18 @@
return null;
}
+ private BufferedFileHandle getOrCreateFileHandle(int fileId) {
+ synchronized (fileInfoMap) {
+ return fileInfoMap.computeIfAbsent(fileId, id -> new BufferedFileHandle(fileId, null));
+ }
+ }
+
+ private BufferedFileHandle removeFileInfo(int fileId) {
+ synchronized (fileInfoMap) {
+ return fileInfoMap.remove(fileId);
+ }
+ }
+
private ICachedPage getPageLoop(long dpid, int multiplier, boolean confiscate) throws HyracksDataException {
final long startingPinCount = DEBUG ? masterPinCount.get() : -1;
int cycleCount = 0;
@@ -1404,14 +1433,14 @@
@Override
public void purgeHandle(int fileId) throws HyracksDataException {
- synchronized (fileInfoMap) {
- BufferedFileHandle fh = fileInfoMap.get(fileId);
- if (fh != null) {
- ioManager.close(fh.getFileHandle());
- fileInfoMap.remove(fileId);
+ BufferedFileHandle fh = removeFileInfo(fileId);
+ if (fh != null) {
+ synchronized (fileInfoMap) {
fileMapManager.unregisterFile(fileId);
}
+ ioManager.close(fh.getFileHandle());
}
+
}
static class BufferCacheHeaderHelper {
@@ -1451,4 +1480,23 @@
return multiplier;
}
}
+
+ @Override
+ public void closeFileIfOpen(FileReference fileRef) {
+ synchronized (fileInfoMap) {
+ if (fileMapManager.isMapped(fileRef)) {
+ int fileId;
+ try {
+ fileId = fileMapManager.lookupFileId(fileRef);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ BufferedFileHandle fInfo = fileInfoMap.get(fileId);
+ if (fInfo != null && fInfo.getReferenceCount() > 0) {
+ fInfo.decReferenceCount();
+ }
+ }
+ }
+ }
+
}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
index 1cbe404..9e92f21 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/DebugBufferCache.java
@@ -246,4 +246,10 @@
deleteFileCount.incrementAndGet();
bufferCache.deleteFile(file);
}
+
+ @Override
+ public void closeFileIfOpen(FileReference fileRef) {
+ bufferCache.closeFileIfOpen(fileRef);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
index 8dccc4a..21d3677 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/IBufferCache.java
@@ -269,4 +269,13 @@
*/
void resizePage(ICachedPage page, int multiplier, IExtraPageBlockHelper extraPageBlockHelper)
throws HyracksDataException;
+
+ /**
+ * Close the file if open.
+ *
+ * @param fileRef
+ * @throws HyracksDataException
+ */
+ void closeFileIfOpen(FileReference fileRef);
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
index 177128e..62e7888 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/file/BufferedFileHandle.java
@@ -37,18 +37,14 @@
return fileId;
}
+ public void setFileHandle(IFileHandle fileHandle) {
+ this.handle = fileHandle;
+ }
+
public IFileHandle getFileHandle() {
return handle;
}
- public void markAsDeleted() {
- handle = null;
- }
-
- public boolean fileHasBeenDeleted() {
- return handle == null;
- }
-
public int incReferenceCount() {
return refCount.incrementAndGet();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
deleted file mode 100644
index d303cca..0000000
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/ITreeIndexTestContext.java
+++ /dev/null
@@ -1,55 +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.hyracks.storage.am.common;
-
-import java.util.Collection;
-
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import org.apache.hyracks.storage.am.common.api.ITreeIndex;
-import org.apache.hyracks.storage.common.IIndexAccessor;
-
-@SuppressWarnings("rawtypes")
-public interface ITreeIndexTestContext<T extends CheckTuple> {
- public int getFieldCount();
-
- public int getKeyFieldCount();
-
- public ISerializerDeserializer[] getFieldSerdes();
-
- public IBinaryComparatorFactory[] getComparatorFactories();
-
- public IIndexAccessor getIndexAccessor();
-
- public ITreeIndex getIndex();
-
- public ArrayTupleReference getTuple();
-
- public ArrayTupleBuilder getTupleBuilder();
-
- public void insertCheckTuple(T checkTuple, Collection<T> checkTuples);
-
- public void deleteCheckTuple(T checkTuple, Collection<T> checkTuples);
-
- public Collection<T> getCheckTuples();
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
new file mode 100644
index 0000000..cb791c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallback.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class EncapsulatingIoCallback implements ILSMIOOperationCallback {
+
+ private final ILSMIOOperationCallback encapsulated;
+ private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+ private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+ private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+ private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+ private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+ public EncapsulatingIoCallback(ILSMIOOperationCallback inner, ITestOpCallback<ILSMIOOperation> scheduledCallback,
+ ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+ ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+ ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+ ITestOpCallback<ILSMIOOperation> completedCallback) {
+ this.encapsulated = inner;
+ this.scheduledCallback = scheduledCallback;
+ this.beforeOperationCallback = beforeOperationCallback;
+ this.afterOperationCallback = afterOperationCallback;
+ this.afterFinalizeCallback = afterFinalizeCallback;
+ this.completedCallback = completedCallback;
+ }
+
+ @Override
+ public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+ scheduledCallback.before(operation);
+ encapsulated.scheduled(operation);
+ scheduledCallback.after(operation);
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+ beforeOperationCallback.before(operation);
+ encapsulated.beforeOperation(operation);
+ beforeOperationCallback.after(operation);
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+ afterOperationCallback.before(operation);
+ encapsulated.afterOperation(operation);
+ afterOperationCallback.after(operation);
+ }
+
+ @Override
+ public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+ afterFinalizeCallback.before(operation);
+ encapsulated.afterFinalize(operation);
+ afterFinalizeCallback.after(operation);
+ }
+
+ @Override
+ public void completed(ILSMIOOperation operation) {
+ try {
+ completedCallback.before(operation);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ encapsulated.completed(operation);
+ try {
+ completedCallback.after(operation);
+ } catch (HyracksDataException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+ encapsulated.recycled(component);
+ }
+
+ @Override
+ public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+ encapsulated.allocated(component);
+ }
+
+ public ILSMIOOperationCallback getEncapsulated() {
+ return encapsulated;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
new file mode 100644
index 0000000..bb10236
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/EncapsulatingIoCallbackFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class EncapsulatingIoCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ private final ILSMIOOperationCallbackFactory encapsulated;
+ private final ITestOpCallback<ILSMIOOperation> scheduledCallback;
+ private final ITestOpCallback<ILSMIOOperation> beforeOperationCallback;
+ private final ITestOpCallback<ILSMIOOperation> afterOperationCallback;
+ private final ITestOpCallback<ILSMIOOperation> afterFinalizeCallback;
+ private final ITestOpCallback<ILSMIOOperation> completedCallback;
+
+ public EncapsulatingIoCallbackFactory(ILSMIOOperationCallbackFactory factory,
+ ITestOpCallback<ILSMIOOperation> scheduledCallback,
+ ITestOpCallback<ILSMIOOperation> beforeOperationCallback,
+ ITestOpCallback<ILSMIOOperation> afterOperationCallback,
+ ITestOpCallback<ILSMIOOperation> afterFinalizeCallback,
+ ITestOpCallback<ILSMIOOperation> completedCallback) {
+ encapsulated = factory;
+ this.scheduledCallback = scheduledCallback;
+ this.beforeOperationCallback = beforeOperationCallback;
+ this.afterOperationCallback = afterOperationCallback;
+ this.afterFinalizeCallback = afterFinalizeCallback;
+ this.completedCallback = completedCallback;
+ }
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) {
+ encapsulated.initialize(ncCtx, resource);
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+ ILSMIOOperationCallback inner = encapsulated.createIoOpCallback(index);
+ return new EncapsulatingIoCallback(inner, scheduledCallback, beforeOperationCallback, afterOperationCallback,
+ afterFinalizeCallback, completedCallback);
+ }
+
+ @Override
+ public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+ return encapsulated.getCurrentMemoryComponentIndex();
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
new file mode 100644
index 0000000..21f3877
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -0,0 +1,470 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.SingleThreadEventProcessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestContext;
+import org.apache.hyracks.storage.am.btree.OrderedIndexTestUtils;
+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.btree.impl.CountingIoOperationCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
+import org.apache.hyracks.storage.am.lsm.btree.impl.NoOpTestCallback;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
+import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.LSMComponentType;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class LSMBTreeComponentLifecycleTest {
+ private static final Logger LOGGER = LogManager.getLogger();
+ private final LSMBTreeTestHarness harness = new LSMBTreeTestHarness();
+ private final OrderedIndexTestUtils testUtils = new OrderedIndexTestUtils();
+ private final ISerializerDeserializer[] fieldSerdes =
+ { IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE };
+ private final int numKeys = 1;
+ private static final int numTuplesToInsert = 100;
+
+ @Before
+ public void setUp() throws HyracksDataException {
+ harness.setUp();
+ }
+
+ @After
+ public void tearDown() throws HyracksDataException {
+ harness.tearDown();
+ }
+
+ private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys,
+ ILSMIOOperationScheduler scheduler, ILSMIOOperationCallbackFactory ioCallbackFactory) throws Exception {
+ return LSMBTreeTestContext.create(harness.getIOManager(), harness.getVirtualBufferCaches(),
+ harness.getFileReference(), harness.getDiskBufferCache(), fieldSerdes, numKeys,
+ harness.getBoomFilterFalsePositiveRate(), harness.getMergePolicy(), harness.getOperationTracker(),
+ scheduler, ioCallbackFactory, harness.getMetadataPageManagerFactory(), false, true, false);
+ }
+
+ private OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys)
+ throws Exception {
+ return createTestContext(fieldSerdes, numKeys, harness.getIOScheduler(),
+ harness.getIOOperationCallbackFactory());
+ }
+
+ @Test
+ public void testFlushUnallocatedIndex() throws Exception {
+ OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+ ILSMIndex index = (ILSMIndex) ctx.getIndex();
+ index.create();
+ index.activate();
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(0, index.getDiskComponents().size());
+ CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+ // assert equal before, after, after were called
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert into the index
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(1, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert more
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(2, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ ctx.getIndex().deactivate();
+ ctx.getIndex().destroy();
+ }
+
+ @Test
+ public void testFlushUnallocatedIndexStartFromSecondComponent() throws Exception {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+ try {
+ testFlushUnallocatedIndex();
+ } finally {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+ }
+ }
+
+ @Test
+ public void testNormalFlushOperation() throws Exception {
+ OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+ ILSMIndex index = (ILSMIndex) ctx.getIndex();
+ index.create();
+ index.activate();
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(1, index.getDiskComponents().size());
+ CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+ // assert equal before, after, after were called
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert into the index
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(2, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert more
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(3, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ ctx.getIndex().deactivate();
+ ctx.getIndex().destroy();
+ }
+
+ @Test
+ public void testNormalFlushOperationStartFromSecondComponent() throws Exception {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+ try {
+ testNormalFlushOperation();
+ } finally {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+ }
+ }
+
+ @Test
+ public void testFlushUnModifiedComponent() throws Exception {
+ OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys);
+ ILSMIndex index = (ILSMIndex) ctx.getIndex();
+ index.create();
+ index.activate();
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(1, index.getDiskComponents().size());
+ CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) index.getIOOperationCallback();
+ // assert equal before, after, finalize were called
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // flush, there was no insert before
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(1, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert more
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(1), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(2, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // insert more
+ testUtils.insertIntTuples(ctx, numTuplesToInsert, harness.getRandom());
+ // flush
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(3, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ ctx.getIndex().deactivate();
+ ctx.getIndex().destroy();
+ }
+
+ @Test
+ public void testFlushUnModifiedComponentStartFromSecondComponent() throws Exception {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 1;
+ try {
+ testFlushUnModifiedComponent();
+ } finally {
+ CountingIoOperationCallbackFactory.STARTING_INDEX = 0;
+ }
+ }
+
+ public int getExpectedMemoryComponentIndex(int expectedIndex) {
+ return (CountingIoOperationCallbackFactory.STARTING_INDEX + expectedIndex) % 2;
+ }
+
+ @Test
+ public void testScheduleMoreFlushesThanComponents() throws Exception {
+ final AtomicInteger counter = new AtomicInteger();
+ final Semaphore flushSemaphore = new Semaphore(0);
+ OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, new AsynchronousScheduler(
+ r -> new Thread(r, "LsmIoThread-" + counter.getAndIncrement()), new IIoOperationFailedCallback() {
+ @Override
+ public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+ LOGGER.log(Level.ERROR, "Scheduler failed", failure);
+ }
+
+ @Override
+ public void operationFailed(ILSMIOOperation operation, Throwable failure) {
+ LOGGER.log(Level.ERROR, "Operation {} failed", operation, failure);
+ }
+ }), new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
+ NoOpTestCallback.get(), new ITestOpCallback<ILSMIOOperation>() {
+ @Override
+ public void before(ILSMIOOperation t) throws HyracksDataException {
+ try {
+ flushSemaphore.acquire();
+ } catch (InterruptedException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ @Override
+ public void after(ILSMIOOperation t) throws HyracksDataException {
+ }
+ }, NoOpTestCallback.get(), NoOpTestCallback.get()));
+ ILSMIndex index = (ILSMIndex) ctx.getIndex();
+ index.create();
+ index.activate();
+ Assert.assertEquals(getExpectedMemoryComponentIndex(0), index.getCurrentMemoryComponentIndex());
+ int numMemoryComponents = index.getNumberOfAllMemoryComponents();
+ // create a flusher that will schedule 13 flushes.
+ // wait for all flushes to be scheduled.
+ // create an inserter that will insert some records.
+ // one by one allow flushes until one flush remains, and ensure no record went in.
+ // allow the last flush, then wait for the inserts to succeed, and ensure they went to
+ // the expected memory component
+ final int numFlushes = 13;
+ User firstUser = new User("FirstUser");
+ User secondUser = new User("SecondUser");
+ Request flushRequest = new Request(Request.Statement.FLUSH, ctx, numFlushes);
+ firstUser.add(flushRequest);
+ firstUser.step();
+ // wait until all flushes have been scheduled.. Not yet performed
+ flushRequest.await(1);
+ // create an inserter and allow it to go all the way
+ Request insertRequest = new Request(Request.Statement.INSERT, ctx, 1);
+ secondUser.add(insertRequest);
+ secondUser.step();
+ secondUser.step();
+ ILSMIndexAccessor accessor = index.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ ILSMIndexOperationContext opCtx = accessor.getOpContext();
+ assertCorrectSearchComponents(opCtx, index, 0);
+ // Allow one flush at a time and ensure that inserter didn't succeed
+ for (int i = 0; i < numFlushes - 1; i++) {
+ flushSemaphore.release();
+ firstUser.step();
+ flushRequest.await(2 + i);
+ Assert.assertEquals(0, insertRequest.getSteps());
+ // also ensure that you get the correct components when searching
+ assertCorrectSearchComponents(opCtx, index, i + 1);
+ }
+ // Allow last flush to proceed
+ flushSemaphore.release();
+ // wait for the insert to complete
+ insertRequest.await();
+ firstUser.step();
+ firstUser.step();
+ flushRequest.await();
+ firstUser.stop();
+ secondUser.stop();
+
+ int expectedMemoryComponent = numFlushes % numMemoryComponents;
+ Assert.assertEquals(getExpectedMemoryComponentIndex(expectedMemoryComponent),
+ index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(0, index.getDiskComponents().size());
+
+ EncapsulatingIoCallback encapsulating = (EncapsulatingIoCallback) index.getIOOperationCallback();
+ CountingIoOperationCallback ioCallback = (CountingIoOperationCallback) encapsulating.getEncapsulated();
+ // assert equal before, after, finalize were called
+ Assert.assertEquals(numFlushes, ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // flush, there was no insert before
+ flushSemaphore.release();
+ flush(ctx);
+ Assert.assertEquals(getExpectedMemoryComponentIndex((expectedMemoryComponent + 1) % numMemoryComponents),
+ index.getCurrentMemoryComponentIndex());
+ Assert.assertEquals(1, index.getDiskComponents().size());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterOperationCount());
+ Assert.assertEquals(ioCallback.getBeforeOperationCount(), ioCallback.getAfterFinalizeCount());
+ // deactivate will cause a flush
+ flushSemaphore.release();
+ ctx.getIndex().deactivate();
+ ctx.getIndex().destroy();
+ }
+
+ private void assertCorrectSearchComponents(ILSMIndexOperationContext opCtx, ILSMIndex index,
+ int numSuccesfullyCompletedFlushes) throws HyracksDataException {
+ opCtx.reset();
+ opCtx.setOperation(IndexOperation.SEARCH);
+ index.getOperationalComponents(opCtx);
+ List<ILSMMemoryComponent> memComponents = index.getMemoryComponents();
+ int first = numSuccesfullyCompletedFlushes % memComponents.size();
+ Assert.assertEquals(memComponents.get(first), getFirstMemoryComponent(opCtx));
+ }
+
+ private ILSMComponent getFirstMemoryComponent(ILSMIndexOperationContext opCtx) {
+ List<ILSMComponent> components = opCtx.getComponentHolder();
+ // backward
+ for (int i = components.size() - 1; i >= 0; i--) {
+ ILSMComponent next = components.get(i);
+ if (next.getType() == LSMComponentType.MEMORY) {
+ return next;
+ }
+ }
+ return null;
+ }
+
+ private void flush(OrderedIndexTestContext ctx) throws HyracksDataException, InterruptedException {
+ ILSMIOOperation flush = scheduleFlush(ctx);
+ flush.sync();
+ if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(flush.getFailure());
+ }
+ }
+
+ private ILSMIOOperation scheduleFlush(OrderedIndexTestContext ctx)
+ throws HyracksDataException, InterruptedException {
+ ILSMIndexAccessor accessor =
+ (ILSMIndexAccessor) ctx.getIndex().createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ return accessor.scheduleFlush();
+ }
+
+ private static class Request {
+ private enum Statement {
+ FLUSH,
+ INSERT
+ }
+
+ private final Statement statement;
+ private final OrderedIndexTestContext ctx;
+ private final int repeats;
+ private boolean done = false;
+ private int step = 0;
+
+ public Request(Statement statement, OrderedIndexTestContext ctx, int repeats) {
+ this.statement = statement;
+ this.ctx = ctx;
+ this.repeats = repeats;
+ }
+
+ Statement statement() {
+ return statement;
+ }
+
+ synchronized void complete() {
+ done = true;
+ notifyAll();
+ }
+
+ synchronized void await() throws InterruptedException {
+ while (!done) {
+ wait();
+ }
+ }
+
+ synchronized void step() {
+ step++;
+ notifyAll();
+ }
+
+ synchronized int getSteps() {
+ return step;
+ }
+
+ synchronized void await(int step) throws InterruptedException {
+ while (this.step < step) {
+ wait();
+ }
+ }
+ }
+
+ private class User extends SingleThreadEventProcessor<Request> {
+
+ private Semaphore step = new Semaphore(0);
+
+ public User(String username) {
+ super(username);
+ }
+
+ public void step() {
+ step.release();
+ }
+
+ @Override
+ protected void handle(Request req) throws Exception {
+ try {
+ step.acquire();
+ switch (req.statement()) {
+ case FLUSH:
+ List<ILSMIOOperation> flushes = new ArrayList<>(req.repeats);
+ for (int i = 0; i < req.repeats; i++) {
+ flushes.add(scheduleFlush(req.ctx));
+ }
+ req.step();
+ for (ILSMIOOperation op : flushes) {
+ step.acquire();
+ op.sync();
+ if (op.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(op.getFailure());
+ }
+ req.step(); // report after completion of each flush
+ }
+ break;
+ case INSERT:
+ testUtils.insertIntTuples(req.ctx, numTuplesToInsert, harness.getRandom());
+ break;
+ default:
+ break;
+ }
+ req.step();
+ step.acquire();
+ } finally {
+ req.step();
+ req.complete();
+ }
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
index 9d768aa..03b221e 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFileManagerTest.java
@@ -29,7 +29,6 @@
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTreeWithBloomFilterDiskComponent;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.junit.After;
import org.junit.Assert;
@@ -71,8 +70,8 @@
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
accessor.insert(tuple);
- // Flush to generate a disk component
- accessor.scheduleFlush(((ILSMIndex) ctx.getIndex()).getIOOperationCallback());
+ // Flush to generate a disk component. This uses synchronous scheduler
+ accessor.scheduleFlush();
// Make sure the disk component was generated
LSMBTree btree = (LSMBTree) ctx.getIndex();
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
index 815c8bc..1b9a7a5 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeFilterMergeTestDriver.java
@@ -40,9 +40,9 @@
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentFilter;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import org.apache.hyracks.storage.am.lsm.common.impls.StubIOOperationCallback;
import org.junit.Assert;
/**
@@ -113,13 +113,14 @@
TreeIndexTestUtils.compareFilterTuples(obsMinMax.getRight(), minMax.getRight(), comp));
}
- StubIOOperationCallback stub = new StubIOOperationCallback();
- BlockingIOOperationCallbackWrapper waiter = new BlockingIOOperationCallbackWrapper(stub);
- accessor.scheduleFlush(waiter);
- waiter.waitForIO();
+ ILSMIOOperation flush = accessor.scheduleFlush();
+ flush.sync();
+ if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(flush.getFailure());
+ }
if (minMax != null) {
Pair<ITupleReference, ITupleReference> obsMinMax =
- filterToMinMax(stub.getLastNewComponent().getLSMComponentFilter());
+ filterToMinMax(flush.getNewComponent().getLSMComponentFilter());
Assert.assertEquals(0,
TreeIndexTestUtils.compareFilterTuples(obsMinMax.getLeft(), minMax.getLeft(), comp));
Assert.assertEquals(0,
@@ -143,8 +144,7 @@
expectedMergeMinMax.setRight(componentMinMax.getRight());
}
}
- accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
- ((LSMBTree) ctx.getIndex()).getDiskComponents());
+ accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
flushedComponents = ((LSMBTree) ctx.getIndex()).getDiskComponents();
Pair<ITupleReference, ITupleReference> mergedMinMax =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
index 475ab9c..a8639c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeFailTest.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexOperationContext;
@@ -76,32 +77,24 @@
ITupleReference tuple1 = TupleUtils.createIntegerTuple(1, 1, 1);
accessor.insert(tuple1);
// flush component
- accessor.scheduleFlush(btree.getIOOperationCallback());
+ accessor.scheduleFlush();
ITupleReference tuple2 = TupleUtils.createIntegerTuple(2, 2, 2);
accessor.insert(tuple2);
// flush component
- accessor.scheduleFlush(btree.getIOOperationCallback());
+ accessor.scheduleFlush();
ITupleReference tuple3 = TupleUtils.createIntegerTuple(3, 3, 3);
accessor.insert(tuple3);
// flush component
- accessor.scheduleFlush(btree.getIOOperationCallback());
-
+ accessor.scheduleFlush();
scheduler.modify = true;
-
- boolean exceptionThrown = false;
- try {
- accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
- } catch (HyracksDataException e) {
- exceptionThrown = true;
- }
- Assert.assertTrue(exceptionThrown);
-
+ ILSMIOOperation merge = accessor.scheduleMerge(btree.getDiskComponents());
+ merge.sync();
+ Assert.assertEquals(LSMIOOperationStatus.FAILURE, merge.getStatus());
scheduler.modify = false;
- accessor.scheduleMerge(btree.getIOOperationCallback(), btree.getDiskComponents());
+ accessor.scheduleMerge(btree.getDiskComponents());
Assert.assertEquals(1, btree.getDiskComponents().size());
-
btree.deactivate();
btree.destroy();
}
@@ -120,14 +113,18 @@
@Override
public void scheduleOperation(ILSMIOOperation operation) throws HyracksDataException {
- if (modify) {
- try {
- modifyOperation(operation);
- } catch (Exception e) {
- throw HyracksDataException.create(e);
+ try {
+ if (modify) {
+ try {
+ modifyOperation(operation);
+ } catch (Exception e) {
+ throw HyracksDataException.create(e);
+ }
}
+ operation.call();
+ } finally {
+ operation.complete();
}
- operation.call();
}
private void modifyOperation(ILSMIOOperation operation) throws Exception {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
index 87ddf33..7c59671 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeMergeTestDriver.java
@@ -74,8 +74,7 @@
}
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
- accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getIOOperationCallback(),
- ((LSMBTree) ctx.getIndex()).getDiskComponents());
+ accessor.scheduleMerge(((LSMBTree) ctx.getIndex()).getDiskComponents());
orderedIndexTestUtils.checkPointSearches(ctx);
orderedIndexTestUtils.checkScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
index 9f17efa..4fafb38 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeModificationOperationCallbackTest.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.storage.am.lsm.btree;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.utils.SerdeUtils;
import org.apache.hyracks.dataflow.common.utils.TupleUtils;
import org.apache.hyracks.storage.am.btree.AbstractModificationOperationCallbackTest;
@@ -26,9 +27,9 @@
import org.apache.hyracks.storage.am.common.impls.NoOpOperationCallback;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
import org.apache.hyracks.util.trace.ITracer;
import org.junit.Test;
@@ -72,8 +73,6 @@
IndexAccessParameters actx = new IndexAccessParameters(cb, NoOpOperationCallback.INSTANCE);
ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(actx);
- BlockingIOOperationCallbackWrapper ioOpCallback =
- new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
for (int j = 0; j < 2; j++) {
isFoundNull = true;
for (int i = 0; i < NUM_TUPLES; i++) {
@@ -82,8 +81,11 @@
}
if (j == 1) {
- accessor.scheduleFlush(ioOpCallback);
- ioOpCallback.waitForIO();
+ ILSMIOOperation flush = accessor.scheduleFlush();
+ flush.sync();
+ if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(flush.getFailure());
+ }
isFoundNull = true;
} else {
isFoundNull = false;
@@ -95,8 +97,7 @@
}
if (j == 1) {
- accessor.scheduleFlush(ioOpCallback);
- ioOpCallback.waitForIO();
+ accessor.scheduleFlush().sync();
isFoundNull = true;
} else {
isFoundNull = false;
@@ -106,9 +107,7 @@
TupleUtils.createIntegerTuple(builder, tuple, i);
accessor.delete(tuple);
}
-
- accessor.scheduleFlush(ioOpCallback);
- ioOpCallback.waitForIO();
+ accessor.scheduleFlush().sync();
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index 48812b4..2574c4d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -44,8 +44,6 @@
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.junit.After;
@@ -113,21 +111,20 @@
protected void test(OrderedIndexTestContext ctx, ISerializerDeserializer[] fieldSerdes)
throws HyracksDataException {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
- ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
//component 2 contains 1 and 2
upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
- accessor.scheduleFlush(ioCallback);
+ accessor.scheduleFlush();
//component 1 contains 1 and -2
upsertTuple(ctx, fieldSerdes, getValue(1, fieldSerdes));
deleteTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
- accessor.scheduleFlush(ioCallback);
+ accessor.scheduleFlush();
//component 0 contains 2 and 3
upsertTuple(ctx, fieldSerdes, getValue(3, fieldSerdes));
upsertTuple(ctx, fieldSerdes, getValue(2, fieldSerdes));
- accessor.scheduleFlush(ioCallback);
+ accessor.scheduleFlush();
LSMBTree btree = (LSMBTree) ctx.getIndex();
Assert.assertEquals("Check disk components", 3, btree.getDiskComponents().size());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index 3a48160..afcb3c1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -52,8 +52,6 @@
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestContext;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.common.IIndexCursor;
import org.junit.After;
@@ -352,11 +350,10 @@
throws HyracksDataException {
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
- ILSMIOOperationCallback ioCallback = ((ILSMIndex) ctx.getIndex()).getIOOperationCallback();
op1.performOperation(ctx, AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT);
op2.performOperation(ctx,
AccessMethodTestsConfig.BTREE_NUM_TUPLES_TO_INSERT / AccessMethodTestsConfig.BTREE_NUM_INSERT_ROUNDS);
- accessor.scheduleFlush(ioCallback);
+ accessor.scheduleFlush();
LSMBTree btree = (LSMBTree) ctx.getIndex();
Assert.assertEquals("Check disk components", 1, btree.getDiskComponents().size());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
index fc852cd..0914541 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceTest.java
@@ -35,9 +35,9 @@
import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
import org.apache.hyracks.storage.am.lsm.btree.util.LSMBTreeTestHarness;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-import org.apache.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerFactory;
import org.apache.hyracks.storage.common.IIndexAccessor;
import org.apache.hyracks.storage.common.IModificationOperationCallback;
@@ -99,8 +99,6 @@
private void test(IndexModification op1, IndexModification op2) throws Exception {
ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) accessor;
- BlockingIOOperationCallbackWrapper ioOpCallback =
- new BlockingIOOperationCallbackWrapper(((ILSMIndex) index).getIOOperationCallback());
for (int j = 0; j < 2; j++) {
index.clear();
isFoundNull = true;
@@ -111,8 +109,11 @@
}
if (j == 1) {
- lsmAccessor.scheduleFlush(ioOpCallback);
- ioOpCallback.waitForIO();
+ ILSMIOOperation flush = lsmAccessor.scheduleFlush();
+ flush.sync();
+ if (flush.getStatus() == LSMIOOperationStatus.FAILURE) {
+ throw HyracksDataException.create(flush.getFailure());
+ }
isFoundNull = true;
isUpdated = false;
} else {
@@ -126,8 +127,7 @@
}
if (j == 1) {
- lsmAccessor.scheduleFlush(ioOpCallback);
- ioOpCallback.waitForIO();
+ lsmAccessor.scheduleFlush().sync();
} else {
isFoundNull = false;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
index 19a9872..2e43198 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/AllowTestOpCallback.java
@@ -35,7 +35,7 @@
}
@Override
- public void after() throws HyracksDataException {
+ public void after(Semaphore t) throws HyracksDataException {
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
new file mode 100644
index 0000000..84b530a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallback.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMMemoryComponent;
+
+public class CountingIoOperationCallback implements ILSMIOOperationCallback {
+ private int beforeOperation;
+ private int afterOperation;
+ private int afterFinalize;
+ private int recycled;
+ private int allocated;
+ private int beforeSchedule;
+ private int destroy;
+
+ public CountingIoOperationCallback() {
+ }
+
+ public int getAfterFinalizeCount() {
+ return afterFinalize;
+ }
+
+ @Override
+ public void recycled(ILSMMemoryComponent component) throws HyracksDataException {
+ recycled++;
+ }
+
+ public int getRecycledCount() {
+ return recycled;
+ }
+
+ @Override
+ public void allocated(ILSMMemoryComponent component) throws HyracksDataException {
+ allocated++;
+ }
+
+ public int getAllocatedCount() {
+ return allocated;
+ }
+
+ @Override
+ public void scheduled(ILSMIOOperation operation) throws HyracksDataException {
+ beforeSchedule++;
+ }
+
+ public int getBeforeScheduleCount() {
+ return beforeSchedule;
+ }
+
+ @Override
+ public void beforeOperation(ILSMIOOperation operation) throws HyracksDataException {
+ beforeOperation++;
+ }
+
+ public int getBeforeOperationCount() {
+ return beforeOperation;
+ }
+
+ @Override
+ public void afterOperation(ILSMIOOperation operation) throws HyracksDataException {
+ afterOperation++;
+ }
+
+ public int getAfterOperationCount() {
+ return afterOperation;
+ }
+
+ @Override
+ public void afterFinalize(ILSMIOOperation operation) throws HyracksDataException {
+ afterFinalize++;
+ }
+
+ @Override
+ public void completed(ILSMIOOperation operation) {
+ destroy++;
+ }
+
+ public int getDestroyCount() {
+ return destroy;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.java
new file mode 100644
index 0000000..b64b9b1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/CountingIoOperationCallbackFactory.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.btree.impl;
+
+import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import org.apache.hyracks.storage.common.IResource;
+
+public class CountingIoOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+ private static final long serialVersionUID = 1L;
+ public static int STARTING_INDEX = 0;
+
+ @Override
+ public void initialize(INCServiceContext ncCtx, IResource resource) {
+ // No op
+ }
+
+ @Override
+ public ILSMIOOperationCallback createIoOpCallback(ILSMIndex index) throws HyracksDataException {
+ return new CountingIoOperationCallback();
+ }
+
+ @Override
+ public int getCurrentMemoryComponentIndex() throws HyracksDataException {
+ return STARTING_INDEX;
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
index e888238..fa3097d 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/ITestOpCallback.java
@@ -23,5 +23,5 @@
public interface ITestOpCallback<T> {
void before(T t) throws HyracksDataException;
- void after() throws HyracksDataException;
+ void after(T t) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
new file mode 100644
index 0000000..2370663
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/NoOpTestCallback.java
@@ -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.
+ */
+package org.apache.hyracks.storage.am.lsm.btree.impl;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@SuppressWarnings("rawtypes")
+public class NoOpTestCallback implements ITestOpCallback {
+ public static final NoOpTestCallback INSTANCE = new NoOpTestCallback();
+
+ private NoOpTestCallback() {
+ }
+
+ @SuppressWarnings("unchecked")
+ public static final <T> ITestOpCallback<T> get() {
+ return INSTANCE;
+ }
+
+ @Override
+ public void before(Object t) throws HyracksDataException {
+ // NoOp
+ }
+
+ @Override
+ public void after(Object t) throws HyracksDataException {
+ // NoOp
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
index 1d4b7d6..4867c71 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestLsmBtree.java
@@ -34,7 +34,6 @@
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponentFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
@@ -65,9 +64,11 @@
private final List<ITestOpCallback<ILSMMemoryComponent>> ioAllocateCallbacks = new ArrayList<>();
private final List<ITestOpCallback<ILSMMemoryComponent>> ioRecycleCallbacks = new ArrayList<>();
+ private final List<ITestOpCallback<Void>> ioOpScheduledCallbacks = new ArrayList<>();
private final List<ITestOpCallback<Void>> ioBeforeCallbacks = new ArrayList<>();
private final List<ITestOpCallback<Void>> ioAfterOpCallbacks = new ArrayList<>();
private final List<ITestOpCallback<Void>> ioAfterFinalizeCallbacks = new ArrayList<>();
+ private final List<ITestOpCallback<Void>> ioOpCompletedCallbacks = new ArrayList<>();
private final List<ITestOpCallback<Void>> allocateComponentCallbacks = new ArrayList<>();
private volatile int numScheduledFlushes;
@@ -110,7 +111,7 @@
super.modify(ictx, tuple);
synchronized (modifyCallbacks) {
for (ITestOpCallback<Semaphore> callback : modifyCallbacks) {
- callback.after();
+ callback.after(modifySemaphore);
}
}
}
@@ -122,17 +123,15 @@
}
@Override
- public void scheduleFlush(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- super.scheduleFlush(ctx, callback);
+ public ILSMIOOperation createFlushOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
numScheduledFlushes++;
+ return super.createFlushOperation(ctx);
}
@Override
- public void scheduleMerge(ILSMIndexOperationContext ctx, ILSMIOOperationCallback callback)
- throws HyracksDataException {
- super.scheduleMerge(ctx, callback);
+ public ILSMIOOperation createMergeOperation(ILSMIndexOperationContext ctx) throws HyracksDataException {
numScheduledMerges++;
+ return super.createMergeOperation(ctx);
}
@Override
@@ -148,7 +147,7 @@
numFinishedFlushes++;
synchronized (flushCallbacks) {
for (ITestOpCallback<Semaphore> callback : flushCallbacks) {
- callback.after();
+ callback.after(flushSemaphore);
}
}
return c;
@@ -167,7 +166,7 @@
numFinishedMerges++;
synchronized (mergeCallbacks) {
for (ITestOpCallback<Semaphore> callback : mergeCallbacks) {
- callback.after();
+ callback.after(mergeSemaphore);
}
}
return c;
@@ -347,6 +346,30 @@
}
}
+ public void addIoScheduledCallback(ITestOpCallback<Void> callback) {
+ synchronized (ioOpScheduledCallbacks) {
+ ioOpScheduledCallbacks.add(callback);
+ }
+ }
+
+ public void clearIoScheduledCallbacks() {
+ synchronized (ioOpScheduledCallbacks) {
+ ioOpScheduledCallbacks.clear();
+ }
+ }
+
+ public void addIoCompletedCallback(ITestOpCallback<Void> callback) {
+ synchronized (ioOpCompletedCallbacks) {
+ ioOpCompletedCallbacks.add(callback);
+ }
+ }
+
+ public void clearIoCompletedCallbacks() {
+ synchronized (ioOpCompletedCallbacks) {
+ ioOpCompletedCallbacks.clear();
+ }
+ }
+
@Override
public void allocateMemoryComponents() throws HyracksDataException {
synchronized (allocateComponentCallbacks) {
@@ -357,7 +380,7 @@
super.allocateMemoryComponents();
synchronized (allocateComponentCallbacks) {
for (ITestOpCallback<Void> callback : allocateComponentCallbacks) {
- callback.after();
+ callback.after(null);
}
}
}
@@ -373,7 +396,7 @@
public void beforeIoOperationReturned() throws HyracksDataException {
synchronized (ioBeforeCallbacks) {
for (ITestOpCallback<Void> callback : ioBeforeCallbacks) {
- callback.after();
+ callback.after(null);
}
}
}
@@ -389,7 +412,7 @@
public void afterIoOperationReturned() throws HyracksDataException {
synchronized (ioAfterOpCallbacks) {
for (ITestOpCallback<Void> callback : ioAfterOpCallbacks) {
- callback.after();
+ callback.after(null);
}
}
}
@@ -405,7 +428,39 @@
public void afterIoFinalizeReturned() throws HyracksDataException {
synchronized (ioAfterFinalizeCallbacks) {
for (ITestOpCallback<Void> callback : ioAfterFinalizeCallbacks) {
- callback.after();
+ callback.after(null);
+ }
+ }
+ }
+
+ public void ioScheduledCalled() throws HyracksDataException {
+ synchronized (ioOpScheduledCallbacks) {
+ for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+ callback.before(null);
+ }
+ }
+ }
+
+ public void ioScheduledReturned() throws HyracksDataException {
+ synchronized (ioOpScheduledCallbacks) {
+ for (ITestOpCallback<Void> callback : ioOpScheduledCallbacks) {
+ callback.after(null);
+ }
+ }
+ }
+
+ public void ioCompletedCalled() throws HyracksDataException {
+ synchronized (ioOpCompletedCallbacks) {
+ for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+ callback.before(null);
+ }
+ }
+ }
+
+ public void ioCompletedReturned() throws HyracksDataException {
+ synchronized (ioOpCompletedCallbacks) {
+ for (ITestOpCallback<Void> callback : ioOpCompletedCallbacks) {
+ callback.after(null);
}
}
}
@@ -421,7 +476,7 @@
public void recycledReturned(ILSMMemoryComponent component) throws HyracksDataException {
synchronized (ioRecycleCallbacks) {
for (ITestOpCallback<ILSMMemoryComponent> callback : ioRecycleCallbacks) {
- callback.after();
+ callback.after(component);
}
}
}
@@ -437,7 +492,7 @@
public void allocatedReturned(ILSMMemoryComponent component) throws HyracksDataException {
synchronized (ioAllocateCallbacks) {
for (ITestOpCallback<ILSMMemoryComponent> callback : ioAllocateCallbacks) {
- callback.after();
+ callback.after(component);
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
index c7e064f..2e7a9f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/impl/TestVirtualBufferCache.java
@@ -212,4 +212,10 @@
public IFileMapManager getFileMapProvider() {
return vbc.getFileMapProvider();
}
+
+ @Override
+ public void closeFileIfOpen(FileReference fileRef) {
+ throw new UnsupportedOperationException();
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 4c07dea..583fd5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -116,7 +116,7 @@
consumeCursorTuples(searchCursor);
break;
case MERGE:
- accessor.scheduleMerge(lsmBTree.getIOOperationCallback(), lsmBTree.getDiskComponents());
+ accessor.scheduleMerge(lsmBTree.getDiskComponents());
break;
default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index a8e0aee..2462c85 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -36,7 +36,9 @@
import org.apache.hyracks.storage.am.common.impls.NoOpIndexAccessParameters;
import org.apache.hyracks.storage.am.lsm.btree.impls.LSMBTree;
import org.apache.hyracks.storage.am.lsm.btree.utils.LSMBTreeUtil;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
@@ -48,10 +50,14 @@
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.test.support.TestStorageManagerComponentHolder;
import org.apache.hyracks.test.support.TestUtils;
+import org.apache.hyracks.util.ExitUtil;
import org.apache.hyracks.util.trace.ITracer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class LSMTreeRunner implements IExperimentRunner {
+ private static final Logger LOGGER = LogManager.getLogger();
private static final int MAX_OPEN_FILES = Integer.MAX_VALUE;
private static final int HYRACKS_FRAME_SIZE = 131072;
@@ -103,8 +109,17 @@
virtualBufferCaches.add(virtualBufferCache);
}
- this.ioScheduler = AsynchronousScheduler.INSTANCE;
- AsynchronousScheduler.INSTANCE.init(threadFactory);
+ this.ioScheduler = new AsynchronousScheduler(threadFactory, new IIoOperationFailedCallback() {
+ @Override
+ public void operationFailed(ILSMIOOperation operation, Throwable t) {
+ LOGGER.error("Operation {} failed", operation, t);
+ }
+
+ @Override
+ public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
+ ExitUtil.exit(ExitUtil.EC_IO_SCHEDULER_FAILED);
+ }
+ });
lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
index 803c5cb..707628b 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/util/LSMBTreeTestHarness.java
@@ -36,14 +36,14 @@
import org.apache.hyracks.storage.am.common.api.IMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.common.freepage.AppendOnlyLinkedMetadataPageManagerFactory;
import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
+import org.apache.hyracks.storage.am.lsm.btree.impl.CountingIoOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -95,12 +95,12 @@
this.memNumPages = AccessMethodTestsConfig.LSM_BTREE_MEM_NUM_PAGES;
this.hyracksFrameSize = AccessMethodTestsConfig.LSM_BTREE_HYRACKS_FRAME_SIZE;
this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_BTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
+ this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
this.mergePolicy = new NoMergePolicy();
this.opTracker = new ThreadCountingTracker();
this.numMutableComponents = AccessMethodTestsConfig.LSM_BTREE_NUM_MUTABLE_COMPONENTS;
this.metadataPageManagerFactory = AppendOnlyLinkedMetadataPageManagerFactory.INSTANCE;
- this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
+ this.ioOpCallbackFactory = new CountingIoOperationCallbackFactory();
}
public void setUp() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 2fd1dbc..54fd3bb 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -30,7 +30,6 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponent.ComponentState;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
@@ -197,7 +196,7 @@
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
- List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(1, List.class);
+ List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
if (mergedSizes != null) {
mergedComponents.forEach(component -> {
mergedSizes.add(component.getComponentSize());
@@ -219,8 +218,7 @@
}
return null;
}
- }).when(accessor).scheduleMerge(Mockito.any(ILSMIOOperationCallback.class),
- Mockito.anyListOf(ILSMDiskComponent.class));
+ }).when(accessor).scheduleMerge(Mockito.anyListOf(ILSMDiskComponent.class));
Mockito.when(index.createAccessor(Mockito.any(IIndexAccessParameters.class))).thenReturn(accessor);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
index 4657109..1a882b8 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/LSMInvertedIndexMergeTest.java
@@ -56,8 +56,7 @@
invIndex.activate();
}
// Perform merge.
- invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
- ((LSMInvertedIndex) invIndex).getDiskComponents());
+ invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
validateAndCheckIndex(testCtx);
runTinySearchWorkload(testCtx, tupleGen);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
index 584de0c..f5a7c9a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/PartitionedLSMInvertedIndexMergeTest.java
@@ -58,8 +58,7 @@
invIndex.activate();
}
// Perform merge.
- invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getIOOperationCallback(),
- ((LSMInvertedIndex) invIndex).getDiskComponents());
+ invIndexAccessor.scheduleMerge(((LSMInvertedIndex) invIndex).getDiskComponents());
validateAndCheckIndex(testCtx);
runTinySearchWorkload(testCtx, tupleGen);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
index a420ba9..da87b27 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/AbstractInvertedIndexTest.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.storage.am.common.datagen.TupleGenerator;
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.impls.LSMInvertedIndex;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
import org.apache.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestContext;
@@ -84,6 +85,12 @@
LSMInvertedIndexTestUtils.compareActualAndExpectedIndexes(testCtx);
}
LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesRangeSearch(testCtx);
+ if (invIndexType == InvertedIndexType.LSM || invIndexType == InvertedIndexType.PARTITIONED_LSM) {
+ LSMInvertedIndex lsmIndex = (LSMInvertedIndex) invIndex;
+ if (!lsmIndex.isMemoryComponentsAllocated() || lsmIndex.isCurrentMutableComponentEmpty()) {
+ LSMInvertedIndexTestUtils.compareActualAndExpectedIndexesMergeSearch(testCtx);
+ }
+ }
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
index 8e41c86..c1e1b4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/common/LSMInvertedIndexTestHarness.java
@@ -43,7 +43,7 @@
import org.apache.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -92,7 +92,7 @@
this.memNumPages = AccessMethodTestsConfig.LSM_INVINDEX_MEM_NUM_PAGES;
this.hyracksFrameSize = AccessMethodTestsConfig.LSM_INVINDEX_HYRACKS_FRAME_SIZE;
this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_INVINDEX_BLOOMFILTER_FALSE_POSITIVE_RATE;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
+ this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
this.mergePolicy = new NoMergePolicy();
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
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 78b1658..6b06f4a 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
@@ -138,7 +138,7 @@
}
case MERGE: {
- accessor.scheduleMerge(invIndex.getIOOperationCallback(), invIndex.getDiskComponents());
+ accessor.scheduleMerge(invIndex.getDiskComponents());
break;
}
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 ae8713c..2435cc4 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
@@ -55,6 +55,9 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.util.LSMInvertedIndexTestUtils.HyracksTaskTestContext;
import org.apache.hyracks.storage.common.IIndex;
import org.apache.hyracks.storage.common.IIndexAccessParameters;
+import org.apache.hyracks.util.trace.ITraceCategoryRegistry;
+import org.apache.hyracks.util.trace.TraceCategoryRegistry;
+import org.apache.hyracks.util.trace.Tracer;
@SuppressWarnings("rawtypes")
public class LSMInvertedIndexTestContext extends OrderedIndexTestContext {
@@ -184,7 +187,9 @@
harness.getMergePolicy(), harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+ invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+ new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+ ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
break;
}
case PARTITIONED_LSM: {
@@ -195,7 +200,9 @@
harness.getOperationTracker(), harness.getIOScheduler(),
harness.getIOOperationCallbackFactory(), invertedIndexFields, filterTypeTraits,
filterCmpFactories, filterFields, filterFieldsForNonBulkLoadOps,
- invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory());
+ invertedIndexFieldsForNonBulkLoadOps, true, harness.getMetadataPageManagerFactory(),
+ new Tracer(LSMInvertedIndexTestContext.class.getSimpleName(),
+ ITraceCategoryRegistry.CATEGORIES_ALL, new TraceCategoryRegistry()));
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 db3bb50..2f58259 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
@@ -19,7 +19,7 @@
package org.apache.hyracks.storage.am.lsm.invertedindex.util;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
import java.io.ByteArrayInputStream;
import java.io.DataInput;
@@ -87,6 +87,8 @@
import org.apache.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
import org.apache.hyracks.storage.am.lsm.invertedindex.api.InvertedListCursor;
import org.apache.hyracks.storage.am.lsm.invertedindex.common.LSMInvertedIndexTestHarness;
+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;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import org.apache.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
@@ -276,17 +278,33 @@
}
}
+ public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
+ throws HyracksDataException {
+ IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+ IInvertedIndexAccessor invIndexAccessor =
+ (IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ compareActualAndExpectedIndexesRangeSearch(testCtx, invIndexAccessor.createRangeSearchCursor());
+ }
+
+ public static void compareActualAndExpectedIndexesMergeSearch(LSMInvertedIndexTestContext testCtx)
+ throws HyracksDataException {
+ IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
+ LSMInvertedIndexAccessor invIndexAccessor =
+ (LSMInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
+ compareActualAndExpectedIndexesRangeSearch(testCtx,
+ new LSMInvertedIndexMergeCursor(invIndexAccessor.getOpContext()));
+ }
+
/**
* Compares actual and expected indexes using the rangeSearch() method of the inverted-index accessor.
*/
- public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx)
- throws HyracksDataException {
+ public static void compareActualAndExpectedIndexesRangeSearch(LSMInvertedIndexTestContext testCtx,
+ IIndexCursor invIndexCursor) throws HyracksDataException {
IInvertedIndex invIndex = (IInvertedIndex) testCtx.getIndex();
int tokenFieldCount = invIndex.getTokenTypeTraits().length;
int invListFieldCount = invIndex.getInvListTypeTraits().length;
IInvertedIndexAccessor invIndexAccessor =
(IInvertedIndexAccessor) invIndex.createAccessor(NoOpIndexAccessParameters.INSTANCE);
- IIndexCursor invIndexCursor = invIndexAccessor.createRangeSearchCursor();
try {
MultiComparator tokenCmp = MultiComparator.create(invIndex.getTokenCmpFactories());
IBinaryComparatorFactory[] tupleCmpFactories =
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
index 0e0f7e5..ee044bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeMergeTestDriver.java
@@ -76,8 +76,7 @@
}
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
- accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getIOOperationCallback(),
- ((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
+ accessor.scheduleMerge(((AbstractLSMRTree) ctx.getIndex()).getDiskComponents());
rTreeTestUtils.checkScan(ctx);
rTreeTestUtils.checkDiskOrderScan(ctx);
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
index 581dbf7..1fa9d95 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/LSMRTreeWithAntiMatterTuplesDeleteTest.java
@@ -25,7 +25,6 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import org.apache.hyracks.storage.am.config.AccessMethodTestsConfig;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndex;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeTestHarness;
import org.apache.hyracks.storage.am.lsm.rtree.util.LSMRTreeWithAntiMatterTuplesTestContext;
@@ -75,9 +74,8 @@
}
protected void flush(AbstractRTreeTestContext ctx) throws HyracksDataException {
- ILSMIndex lsmIndex = (ILSMIndex) ctx.getIndex();
ILSMIndexAccessor accessor = (ILSMIndexAccessor) ctx.getIndexAccessor();
- accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
+ accessor.scheduleFlush();
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
index 2a14780..810c3c3 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java
@@ -78,7 +78,7 @@
break;
case MERGE:
- accessor.scheduleMerge(lsmRTree.getIOOperationCallback(), lsmRTree.getDiskComponents());
+ accessor.scheduleMerge(lsmRTree.getDiskComponents());
break;
default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
index 653677c..595d675 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java
@@ -66,8 +66,7 @@
break;
case MERGE:
- accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getIOOperationCallback(),
- ((AbstractLSMRTree) lsmRTree).getDiskComponents());
+ accessor.scheduleMerge(((AbstractLSMRTree) lsmRTree).getDiskComponents());
break;
default:
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
index 8a5d0c5..598cf18 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/util/LSMRTreeTestHarness.java
@@ -42,7 +42,7 @@
import org.apache.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
import org.apache.hyracks.storage.am.lsm.common.impls.NoMergePolicy;
import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallbackFactory;
-import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
import org.apache.hyracks.storage.am.lsm.common.impls.ThreadCountingTracker;
import org.apache.hyracks.storage.am.lsm.common.impls.VirtualBufferCache;
import org.apache.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -89,7 +89,7 @@
this.memNumPages = AccessMethodTestsConfig.LSM_RTREE_MEM_NUM_PAGES;
this.bloomFilterFalsePositiveRate = AccessMethodTestsConfig.LSM_RTREE_BLOOMFILTER_FALSE_POSITIVE_RATE;
this.hyracksFrameSize = AccessMethodTestsConfig.LSM_RTREE_HYRACKS_FRAME_SIZE;
- this.ioScheduler = SynchronousScheduler.INSTANCE;
+ this.ioScheduler = SynchronousSchedulerProvider.INSTANCE.getIoScheduler(null);
this.mergePolicy = new NoMergePolicy();
this.opTracker = new ThreadCountingTracker();
this.ioOpCallbackFactory = NoOpIOOperationCallbackFactory.INSTANCE;
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index 212673e..ff5f622 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -79,10 +79,22 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-core</artifactId>
</dependency>
- <dependency>
- <groupId>commons-io</groupId>
- <artifactId>commons-io</artifactId>
- </dependency>
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-text</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.httpcomponents</groupId>
+ <artifactId>httpcore</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.httpcomponents</groupId>
+ <artifactId>httpclient</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 1a17012..4aa123b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -37,12 +37,11 @@
public static final int EC_UNHANDLED_EXCEPTION = 11;
public static final int EC_IMMEDIATE_HALT = 33;
public static final int EC_HALT_ABNORMAL_RESERVED_44 = 44;
- public static final int EC_HALT_ABNORMAL_RESERVED_55 = 55;
+ public static final int EC_IO_SCHEDULER_FAILED = 55;
public static final int EC_HALT_SHUTDOWN_TIMED_OUT = 66;
public static final int EC_HALT_WATCHDOG_FAILED = 77;
- public static final int EC_HALT_ABNORMAL_RESERVED_88 = 88;
+ public static final int EC_FLUSH_FAILED = 88;
public static final int EC_TERMINATE_NC_SERVICE_DIRECTIVE = 99;
-
private static final ExitThread exitThread = new ExitThread();
private static final ShutdownWatchdog watchdogThread = new ShutdownWatchdog();
private static final MutableLong shutdownHaltDelay = new MutableLong(10 * 60 * 1000L); // 10 minutes default
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
index 6085c1c..baa3174 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/JSONUtil.java
@@ -149,7 +149,7 @@
return quoteAndEscape(new StringBuilder(), str).toString();
}
- private static StringBuilder quoteAndEscape(StringBuilder sb, String str) {
+ public static StringBuilder quoteAndEscape(StringBuilder sb, String str) {
return escape(sb.append('"'), str).append('"');
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
index 2b65106..2545c25 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
@@ -64,27 +64,27 @@
return getOpenFileDescriptorCount != null;
}
- public static Integer getCurrentOpenFileCount() {
+ public static Long getCurrentOpenFileCount() {
if (getOpenFileDescriptorCount == null) {
- return -1;
+ return -1L;
}
try {
- return (Integer) getOpenFileDescriptorCount.invoke(osMXBean);
+ return (Long) getOpenFileDescriptorCount.invoke(osMXBean);
} catch (Throwable e) { // NOSONAR
LOGGER.log(Level.WARN, "Failure invoking getOpenFileDescriptorCount", e);
- return -1;
+ return -1L;
}
}
- public static Integer getMaxOpenFileCount() {
+ public static Long getMaxOpenFileCount() {
if (getMaxFileDescriptorCount == null) {
- return -1;
+ return -1L;
}
try {
- return (Integer) getMaxFileDescriptorCount.invoke(osMXBean);
+ return (Long) getMaxFileDescriptorCount.invoke(osMXBean);
} catch (Throwable e) { // NOSONAR
LOGGER.log(Level.WARN, "Failure invoking getMaxFileDescriptorCount", e);
- return -1;
+ return -1L;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
index f9f45c1..1863b28 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
@@ -18,10 +18,19 @@
*/
package org.apache.hyracks.util;
+import java.io.Closeable;
import java.io.IOException;
+import java.net.InetSocketAddress;
import java.net.StandardSocketOptions;
+import java.net.URI;
+import java.net.URISyntaxException;
import java.nio.channels.SocketChannel;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.http.HttpHost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.conn.util.InetAddressUtils;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -37,13 +46,65 @@
sc.setOption(StandardSocketOptions.SO_KEEPALIVE, true);
}
- public static void closeQuietly(SocketChannel sc) {
- if (sc.isOpen()) {
+ public static void closeQuietly(Closeable closeable) {
+ if (closeable != null) {
try {
- sc.close();
+ closeable.close();
} catch (IOException e) {
- LOGGER.warn("Failed to close socket", e);
+ LOGGER.warn("Failed to close", e);
}
}
}
+
+ public static URI toUri(HttpHost host) throws URISyntaxException {
+ return builderFrom(host).build();
+ }
+
+ public static URI toUri(HttpHost host, String path) throws URISyntaxException {
+ return builderFrom(host).setPath(path).build();
+ }
+
+ public static URIBuilder builderFrom(HttpHost host) {
+ return new URIBuilder().setHost(host.getHostName()).setPort(host.getPort()).setScheme(host.getSchemeName());
+ }
+
+ public static String toHostPort(String host, String port) {
+ return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+ }
+
+ public static String toHostPort(String host, int port) {
+ return InetAddressUtils.isIPv6Address(host) ? "[" + host + "]:" + port : host + ":" + port;
+ }
+
+ public static String toHostPort(InetSocketAddress address) {
+ return toHostPort(address.getHostString(), address.getPort());
+ }
+
+ public static InetSocketAddress parseInetSocketAddress(String hostPortString) {
+ int lastColon = hostPortString.lastIndexOf(':');
+ String host = decodeIPv6LiteralHost(lastColon < 0 ? hostPortString : hostPortString.substring(0, lastColon));
+ int port = lastColon < 0 ? 0 : Integer.parseInt(hostPortString.substring(lastColon + 1));
+ return InetSocketAddress.createUnresolved(host, port);
+ }
+
+ public static InetSocketAddress toInetSocketAddress(String maybeLiteralHost, int port) {
+ return InetSocketAddress.createUnresolved(decodeIPv6LiteralHost(maybeLiteralHost), port);
+ }
+
+ public static List<InetSocketAddress> parseInetSocketAddresses(String... hostPortStrings) {
+ List<InetSocketAddress> hosts = new ArrayList<>();
+ for (String node : hostPortStrings) {
+ hosts.add(parseInetSocketAddress(node));
+ }
+ return hosts;
+ }
+
+ public static String encodeIPv6LiteralHost(String hostname) {
+ return InetAddressUtils.isIPv6Address(hostname) ? "[" + hostname + "]" : hostname;
+ }
+
+ public static String decodeIPv6LiteralHost(String hostname) {
+ return hostname.length() > 0 && hostname.charAt(0) == '[' ? hostname.substring(1, hostname.length() - 1)
+ : hostname;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
index b457c79..11be0ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StringUtil.java
@@ -23,7 +23,7 @@
import java.util.regex.Pattern;
import org.apache.commons.collections4.map.LRUMap;
-import org.apache.commons.lang3.text.WordUtils;
+import org.apache.commons.text.WordUtils;
public class StringUtil {
private static final Map<String, String> CAMEL_CACHE = Collections.synchronizedMap(new LRUMap<>(1024));
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
new file mode 100644
index 0000000..e092666
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/annotations/CriticalPath.java
@@ -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.
+ */
+package org.apache.hyracks.util.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * The method on which this annotation is applied has a very high call frequency during normal execution.
+ * Careful consideration should be taken with regard to the number of objects created and any expensive operations in
+ * there.
+ */
+@Documented
+@Target({ ElementType.METHOD })
+@Retention(RetentionPolicy.SOURCE)
+public @interface CriticalPath {
+}
\ No newline at end of file
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 11fb6c0..cdb82bb 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
@@ -84,6 +84,11 @@
throw new IllegalStateException();
}
+ public static boolean isCharStart(byte[] b, int s) {
+ int c = b[s] & 0xff;
+ return (c >> 6) != 2;
+ }
+
public static int getModifiedUTF8Len(char c) {
if (c >= 0x0001 && c <= 0x007F) {
return 1;
@@ -595,7 +600,7 @@
out.write(tempBytes, 0, count);
}
- static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
+ public static void writeUTF8(char[] buffer, int start, int length, DataOutput out, UTF8StringWriter writer)
throws IOException {
int utflen = 0;
int count = 0;
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 62407a4..d553c3a 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -128,6 +128,11 @@
<version>2.6</version>
</dependency>
<dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-text</artifactId>
+ <version>1.2</version>
+ </dependency>
+ <dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<version>2.8.4</version>