[NO ISSUE] Added Optimized Interval Joins
- user model changes: no
- storage format changes: no
- interface changes: no
details:
-Support for optimized interval joins over the 7 of the 13 Allen's relations: Before, After, Covers, Covered_by, Overlaps, Overlapping, and Overlapped_by. Starts, Started_by, ends, ended_by, meets, and met_by are not yet implemented and will default to Hybrid Hash Join.
-Adds a RangeHint
-Updates Optimizer tests for interval joins
-Updates interval join runtime tests to include range hints, tests for time and datetime, and a new data point that tests for additional cases
-Support for PartialBroadcastRangeFollowingExchange and PartialBroadcastRangeIntersectExchange physical operators
-Adds IntervalMergeJoinPOperator and a IntervalMergeJoin Operator Descriptor with a IntervalMergeJoin algorithm and supporting files
-Adds a RunFileStream, RunFilePointer, and a TupleAccessor
Change-Id: Ib90a953881461f8574f1170ad2264b95fa81fd4b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/7163
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalMergeJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalMergeJoinPOperator.java
new file mode 100644
index 0000000..4815349
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalMergeJoinPOperator.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.optimizer.rules.util.IntervalPartitions;
+import org.apache.asterix.runtime.operators.joins.interval.IntervalMergeJoinOperatorDescriptor;
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtilFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedFollowingProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedIntersectProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class IntervalMergeJoinPOperator extends AbstractJoinPOperator {
+
+ private final List<LogicalVariable> keysLeftBranch;
+ private final List<LogicalVariable> keysRightBranch;
+ protected final IIntervalJoinUtilFactory mjcf;
+ protected final IntervalPartitions intervalPartitions;
+
+ private final int memSizeInFrames;
+
+ public IntervalMergeJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+ List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities,
+ int memSizeInFrames, IIntervalJoinUtilFactory mjcf, IntervalPartitions intervalPartitions) {
+ super(kind, partitioningType);
+ this.keysLeftBranch = sideLeftOfEqualities;
+ this.keysRightBranch = sideRightOfEqualities;
+ this.mjcf = mjcf;
+ this.intervalPartitions = intervalPartitions;
+ this.memSizeInFrames = memSizeInFrames;
+ }
+
+ public IIntervalJoinUtilFactory getIntervalMergeJoinCheckerFactory() {
+ return mjcf;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.INTERVAL_MERGE_JOIN;
+ }
+
+ @Override
+ public String toString() {
+ return "INTERVAL_MERGE_JOIN" + " " + keysLeftBranch + " " + keysRightBranch;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context) {
+ List<OrderColumn> order = intervalPartitions.getLeftStartColumn();
+ IPartitioningProperty pp = new OrderedPartitionedProperty(order, null, intervalPartitions.getRangeMap());
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<>();
+ propsLocal.add(new LocalOrderProperty(intervalPartitions.getLeftStartColumn()));
+ deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+ IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ //Create Left Local Order Column
+ IPartitioningProperty ppLeft = null;
+ List<ILocalStructuralProperty> ispLeft = new ArrayList<>();
+ ArrayList<OrderColumn> leftLocalOrderColumn = new ArrayList<>();
+ for (LogicalVariable v : keysLeftBranch) {
+ leftLocalOrderColumn.add(new OrderColumn(v, intervalPartitions.getLeftIntervalColumn().get(0).getOrder()));
+ }
+ ispLeft.add(new LocalOrderProperty(leftLocalOrderColumn));
+
+ //Create Right Local Order Column
+ IPartitioningProperty ppRight = null;
+ List<ILocalStructuralProperty> ispRight = new ArrayList<>();
+ ArrayList<OrderColumn> rightLocalOrderColumn = new ArrayList<>();
+ for (LogicalVariable v : keysRightBranch) {
+ rightLocalOrderColumn
+ .add(new OrderColumn(v, intervalPartitions.getRightIntervalColumn().get(0).getOrder()));
+ }
+ ispRight.add(new LocalOrderProperty(rightLocalOrderColumn));
+
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ INodeDomain targetNodeDomain = context.getComputationNodeDomain();
+
+ RangeMap rangeMapHint = intervalPartitions.getRangeMap();
+
+ //Assign Property
+ switch (intervalPartitions.getLeftPartitioningType()) {
+ case ORDERED_PARTITIONED:
+ ppLeft = new OrderedPartitionedProperty(intervalPartitions.getLeftStartColumn(), targetNodeDomain,
+ rangeMapHint);
+ break;
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING:
+ ppLeft = new PartialBroadcastOrderedFollowingProperty(intervalPartitions.getLeftStartColumn(),
+ targetNodeDomain, rangeMapHint);
+ break;
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT:
+ ppLeft = new PartialBroadcastOrderedIntersectProperty(intervalPartitions.getLeftIntervalColumn(),
+ targetNodeDomain, rangeMapHint);
+ break;
+ }
+ switch (intervalPartitions.getRightPartitioningType()) {
+ case ORDERED_PARTITIONED:
+ ppRight = new OrderedPartitionedProperty(intervalPartitions.getRightStartColumn(), targetNodeDomain,
+ rangeMapHint);
+ break;
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING:
+ ppRight = new PartialBroadcastOrderedFollowingProperty(intervalPartitions.getRightStartColumn(),
+ targetNodeDomain, rangeMapHint);
+ break;
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT:
+ ppRight = new PartialBroadcastOrderedIntersectProperty(intervalPartitions.getRightIntervalColumn(),
+ targetNodeDomain, rangeMapHint);
+ break;
+ }
+ }
+ pv[0] = new StructuralPropertiesVector(ppLeft, ispLeft);
+ pv[1] = new StructuralPropertiesVector(ppRight, ispRight);
+ IPartitioningRequirementsCoordinator prc = IPartitioningRequirementsCoordinator.NO_COORDINATION;
+ return new PhysicalRequirements(pv, prc);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int[] keysLeft = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+ int[] keysRight = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+
+ IOperatorDescriptorRegistry spec = builder.getJobSpec();
+ RecordDescriptor recordDescriptor =
+ JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
+
+ IOperatorDescriptor opDesc = getIntervalOperatorDescriptor(keysLeft, keysRight, spec, recordDescriptor, mjcf);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getValue();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+
+ IOperatorDescriptor getIntervalOperatorDescriptor(int[] keysLeft, int[] keysRight, IOperatorDescriptorRegistry spec,
+ RecordDescriptor recordDescriptor, IIntervalJoinUtilFactory mjcf) {
+ return new IntervalMergeJoinOperatorDescriptor(spec, memSizeInFrames, keysLeft, keysRight, recordDescriptor,
+ mjcf);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 31de7ee..e662737 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -34,6 +34,7 @@
import org.apache.asterix.optimizer.base.AnalysisUtil;
import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
import org.apache.asterix.optimizer.rules.am.BTreeJobGenParams;
+import org.apache.asterix.optimizer.rules.util.AsterixJoinUtils;
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -53,6 +54,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterUnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
@@ -79,6 +82,26 @@
}
@Override
+ public IPhysicalOperator visitInnerJoinOperator(InnerJoinOperator op, Boolean topLevelOp)
+ throws AlgebricksException {
+ AsterixJoinUtils.setJoinAlgorithmAndExchangeAlgo(op, topLevelOp, context);
+ if (op.getPhysicalOperator() != null) {
+ return op.getPhysicalOperator();
+ }
+ return super.visitInnerJoinOperator(op, topLevelOp);
+ }
+
+ @Override
+ public IPhysicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean topLevelOp)
+ throws AlgebricksException {
+ AsterixJoinUtils.setJoinAlgorithmAndExchangeAlgo(op, topLevelOp, context);
+ if (op.getPhysicalOperator() != null) {
+ return op.getPhysicalOperator();
+ }
+ return super.visitLeftOuterJoinOperator(op, topLevelOp);
+ }
+
+ @Override
public ExternalGroupByPOperator createExternalGroupByPOperator(GroupByOperator gby) throws AlgebricksException {
Mutable<ILogicalOperator> r0 = gby.getNestedPlans().get(0).getRoots().get(0);
if (!r0.getValue().getOperatorTag().equals(LogicalOperatorTag.AGGREGATE)) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
new file mode 100644
index 0000000..45b8b07
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.annotations.RangeAnnotation;
+import org.apache.asterix.om.types.ATypeTag;
+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.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class AsterixJoinUtils {
+
+ private static final int LEFT = 0;
+ private static final int RIGHT = 1;
+
+ private AsterixJoinUtils() {
+ }
+
+ public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, Boolean topLevelOp,
+ IOptimizationContext context) throws AlgebricksException {
+ if (!topLevelOp) {
+ return;
+ }
+ ILogicalExpression conditionLE = op.getCondition().getValue();
+ if (conditionLE.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return;
+ }
+ List<LogicalVariable> sideLeft = new ArrayList<>(1);
+ List<LogicalVariable> sideRight = new ArrayList<>(1);
+ List<LogicalVariable> varsLeft = op.getInputs().get(LEFT).getValue().getSchema();
+ List<LogicalVariable> varsRight = op.getInputs().get(RIGHT).getValue().getSchema();
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) conditionLE;
+ FunctionIdentifier fi =
+ IntervalJoinUtils.isIntervalJoinCondition(fexp, varsLeft, varsRight, sideLeft, sideRight, LEFT, RIGHT);
+ if (fi == null) {
+ return;
+ }
+ RangeAnnotation rangeAnnotation = IntervalJoinUtils.findRangeAnnotation(fexp);
+ if (rangeAnnotation == null) {
+ return;
+ }
+ //Check RangeMap type
+ RangeMap rangeMap = (RangeMap) rangeAnnotation.getObject();
+ if (rangeMap.getTag(0, 0) != ATypeTag.DATETIME.serialize() && rangeMap.getTag(0, 0) != ATypeTag.DATE.serialize()
+ && rangeMap.getTag(0, 0) != ATypeTag.TIME.serialize()) {
+ IWarningCollector warningCollector = context.getWarningCollector();
+ warningCollector.warn(Warning.forHyracks(op.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "Date, DateTime, and Time are only range hints types supported for interval joins"));
+ return;
+ }
+ IntervalPartitions intervalPartitions =
+ IntervalJoinUtils.createIntervalPartitions(op, fi, sideLeft, sideRight, rangeMap, context, LEFT, RIGHT);
+ IntervalJoinUtils.setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, context, intervalPartitions);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
new file mode 100644
index 0000000..d887914
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
@@ -0,0 +1,251 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.algebra.operators.physical.IntervalMergeJoinPOperator;
+import org.apache.asterix.common.annotations.RangeAnnotation;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.runtime.operators.joins.interval.utils.AfterIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.BeforeIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.CoveredByIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.CoversIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.OverlappedByIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.OverlappingIntervalJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.interval.utils.OverlapsIntervalJoinUtilFactory;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
+import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class IntervalJoinUtils {
+
+ private static final Map<FunctionIdentifier, FunctionIdentifier> INTERVAL_JOIN_CONDITIONS = new HashMap<>();
+
+ static {
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_AFTER, BuiltinFunctions.INTERVAL_BEFORE);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_BEFORE, BuiltinFunctions.INTERVAL_AFTER);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_COVERED_BY, BuiltinFunctions.INTERVAL_COVERS);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_COVERS, BuiltinFunctions.INTERVAL_COVERED_BY);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_OVERLAPPED_BY, BuiltinFunctions.INTERVAL_OVERLAPS);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_OVERLAPPING, BuiltinFunctions.INTERVAL_OVERLAPPING);
+ INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_OVERLAPS, BuiltinFunctions.INTERVAL_OVERLAPPED_BY);
+ }
+
+ protected static RangeAnnotation findRangeAnnotation(AbstractFunctionCallExpression fexp) {
+ Iterator<IExpressionAnnotation> annotationIter = fexp.getAnnotations().values().iterator();
+ while (annotationIter.hasNext()) {
+ IExpressionAnnotation annotation = annotationIter.next();
+ if (annotation instanceof RangeAnnotation) {
+ return (RangeAnnotation) annotation;
+ }
+ }
+ return null;
+ }
+
+ protected static void setSortMergeIntervalJoinOp(AbstractBinaryJoinOperator op, FunctionIdentifier fi,
+ List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IOptimizationContext context,
+ IntervalPartitions intervalPartitions) throws CompilationException {
+ IIntervalJoinUtilFactory mjcf = createIntervalJoinCheckerFactory(fi, intervalPartitions.getRangeMap());
+ op.setPhysicalOperator(new IntervalMergeJoinPOperator(op.getJoinKind(),
+ AbstractJoinPOperator.JoinPartitioningType.BROADCAST, sideLeft, sideRight,
+ context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), mjcf, intervalPartitions));
+ }
+
+ /**
+ * Certain Relations not yet supported as seen below. Will default to regular join.
+ * Inserts partition sort key.
+ */
+ protected static IntervalPartitions createIntervalPartitions(AbstractBinaryJoinOperator op, FunctionIdentifier fi,
+ List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, RangeMap rangeMap,
+ IOptimizationContext context, int left, int right) throws AlgebricksException {
+
+ List<LogicalVariable> leftPartitionVar = new ArrayList<>(2);
+ leftPartitionVar.add(context.newVar());
+ leftPartitionVar.add(context.newVar());
+ List<LogicalVariable> rightPartitionVar = new ArrayList<>(2);
+ rightPartitionVar.add(context.newVar());
+ rightPartitionVar.add(context.newVar());
+
+ insertPartitionSortKey(op, left, leftPartitionVar, sideLeft.get(0), context);
+ insertPartitionSortKey(op, right, rightPartitionVar, sideRight.get(0), context);
+
+ List<IntervalColumn> leftIC = Collections.singletonList(new IntervalColumn(leftPartitionVar.get(0),
+ leftPartitionVar.get(1), OrderOperator.IOrder.OrderKind.ASC));
+ List<IntervalColumn> rightIC = Collections.singletonList(new IntervalColumn(rightPartitionVar.get(0),
+ rightPartitionVar.get(1), OrderOperator.IOrder.OrderKind.ASC));
+
+ //Set Partitioning Types
+ PartitioningType leftPartitioningType = PartitioningType.ORDERED_PARTITIONED;
+ PartitioningType rightPartitioningType = PartitioningType.ORDERED_PARTITIONED;
+ if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+ rightPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPS)) {
+ leftPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPPING)) {
+ leftPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ rightPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_COVERS)) {
+ leftPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_COVERED_BY)) {
+ rightPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_BEFORE)) {
+ leftPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_FOLLOWING;
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_AFTER)) {
+ rightPartitioningType = PartitioningType.PARTIAL_BROADCAST_ORDERED_FOLLOWING;
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fi.getName());
+ }
+ return new IntervalPartitions(rangeMap, leftIC, rightIC, leftPartitioningType, rightPartitioningType);
+ }
+
+ protected static FunctionIdentifier isIntervalJoinCondition(ILogicalExpression e,
+ Collection<LogicalVariable> inLeftAll, Collection<LogicalVariable> inRightAll,
+ Collection<LogicalVariable> outLeftFields, Collection<LogicalVariable> outRightFields, int left,
+ int right) {
+ FunctionIdentifier fiReturn;
+ boolean switchArguments = false;
+ if (e.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return null;
+ }
+ AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+ FunctionIdentifier fi = fexp.getFunctionIdentifier();
+ if (isIntervalFunction(fi)) {
+ fiReturn = fi;
+ } else {
+ return null;
+ }
+ ILogicalExpression opLeft = fexp.getArguments().get(left).getValue();
+ ILogicalExpression opRight = fexp.getArguments().get(right).getValue();
+ if (opLeft.getExpressionTag() != LogicalExpressionTag.VARIABLE
+ || opRight.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ return null;
+ }
+ LogicalVariable var1 = ((VariableReferenceExpression) opLeft).getVariableReference();
+ if (inLeftAll.contains(var1) && !outLeftFields.contains(var1)) {
+ outLeftFields.add(var1);
+ } else if (inRightAll.contains(var1) && !outRightFields.contains(var1)) {
+ outRightFields.add(var1);
+ fiReturn = getInverseIntervalFunction(fi);
+ switchArguments = true;
+ } else {
+ return null;
+ }
+ LogicalVariable var2 = ((VariableReferenceExpression) opRight).getVariableReference();
+ if (inLeftAll.contains(var2) && !outLeftFields.contains(var2) && switchArguments) {
+ outLeftFields.add(var2);
+ } else if (inRightAll.contains(var2) && !outRightFields.contains(var2) && !switchArguments) {
+ outRightFields.add(var2);
+ } else {
+ return null;
+ }
+ return fiReturn;
+ }
+
+ /**
+ * Certain Relations not yet supported as seen below. Will default to regular join.
+ */
+ private static IIntervalJoinUtilFactory createIntervalJoinCheckerFactory(FunctionIdentifier fi, RangeMap rangeMap)
+ throws CompilationException {
+ IIntervalJoinUtilFactory mjcf;
+ if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+ mjcf = new OverlappedByIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPS)) {
+ mjcf = new OverlapsIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_COVERS)) {
+ mjcf = new CoversIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_COVERED_BY)) {
+ mjcf = new CoveredByIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_BEFORE)) {
+ mjcf = new BeforeIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_AFTER)) {
+ mjcf = new AfterIntervalJoinUtilFactory();
+ } else if (fi.equals(BuiltinFunctions.INTERVAL_OVERLAPPING)) {
+ mjcf = new OverlappingIntervalJoinUtilFactory(rangeMap);
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, fi.getName());
+ }
+ return mjcf;
+ }
+
+ private static boolean isIntervalFunction(FunctionIdentifier fi) {
+ return INTERVAL_JOIN_CONDITIONS.containsKey(fi);
+ }
+
+ private static FunctionIdentifier getInverseIntervalFunction(FunctionIdentifier fi) {
+ return INTERVAL_JOIN_CONDITIONS.get(fi);
+ }
+
+ private static void insertPartitionSortKey(AbstractBinaryJoinOperator op, int branch,
+ List<LogicalVariable> partitionVars, LogicalVariable intervalVar, IOptimizationContext context)
+ throws AlgebricksException {
+ Mutable<ILogicalExpression> intervalExp = new MutableObject<>(new VariableReferenceExpression(intervalVar));
+
+ List<Mutable<ILogicalExpression>> assignExps = new ArrayList<>();
+ // Start partition
+ IFunctionInfo startFi = FunctionUtil.getFunctionInfo(BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START);
+ ScalarFunctionCallExpression startPartitionExp = new ScalarFunctionCallExpression(startFi, intervalExp);
+ assignExps.add(new MutableObject<>(startPartitionExp));
+ // End partition
+ IFunctionInfo endFi = FunctionUtil.getFunctionInfo(BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END);
+ ScalarFunctionCallExpression endPartitionExp = new ScalarFunctionCallExpression(endFi, intervalExp);
+ assignExps.add(new MutableObject<>(endPartitionExp));
+
+ AssignOperator ao = new AssignOperator(partitionVars, assignExps);
+ ao.setSourceLocation(op.getSourceLocation());
+ ao.setExecutionMode(op.getExecutionMode());
+ AssignPOperator apo = new AssignPOperator();
+ ao.setPhysicalOperator(apo);
+ Mutable<ILogicalOperator> aoRef = new MutableObject<>(ao);
+ ao.getInputs().add(op.getInputs().get(branch));
+ op.getInputs().set(branch, aoRef);
+
+ context.computeAndSetTypeEnvironmentForOperator(ao);
+ }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalPartitions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalPartitions.java
new file mode 100644
index 0000000..bc78017
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalPartitions.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
+import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class IntervalPartitions {
+
+ private final RangeMap rangeMap;
+ private final List<IntervalColumn> leftIntervalColumn;
+ private final List<IntervalColumn> rightIntervalColumn;
+ private final PartitioningType leftPartitioningType;
+ private final PartitioningType rightPartitioningType;
+
+ IntervalPartitions(RangeMap rangeMap, List<IntervalColumn> leftIntervalColumn,
+ List<IntervalColumn> rightIntervalColumn, PartitioningType leftPartitioningType,
+ PartitioningType rightPartitioningType) {
+ this.rangeMap = rangeMap;
+ this.leftIntervalColumn = leftIntervalColumn;
+ this.rightIntervalColumn = rightIntervalColumn;
+ this.leftPartitioningType = leftPartitioningType;
+ this.rightPartitioningType = rightPartitioningType;
+ }
+
+ public RangeMap getRangeMap() {
+ return rangeMap;
+ }
+
+ public PartitioningType getLeftPartitioningType() {
+ return leftPartitioningType;
+ }
+
+ public PartitioningType getRightPartitioningType() {
+ return rightPartitioningType;
+ }
+
+ public List<IntervalColumn> getLeftIntervalColumn() {
+ return leftIntervalColumn;
+ }
+
+ public List<IntervalColumn> getRightIntervalColumn() {
+ return rightIntervalColumn;
+ }
+
+ public List<OrderColumn> getLeftStartColumn() {
+ LogicalVariable leftStartLogicalVariable = leftIntervalColumn.get(0).getStartColumn();
+ List<OrderColumn> leftOrderColumn =
+ Arrays.asList(new OrderColumn(leftStartLogicalVariable, leftIntervalColumn.get(0).getOrder()));
+ return leftOrderColumn;
+ }
+
+ public List<OrderColumn> getRightStartColumn() {
+ LogicalVariable rightStartLogicalVariable = rightIntervalColumn.get(0).getStartColumn();
+ List<OrderColumn> rightOrderColumn =
+ Arrays.asList(new OrderColumn(rightStartLogicalVariable, rightIntervalColumn.get(0).getOrder()));
+ return rightOrderColumn;
+ }
+}
diff --git a/asterixdb/asterix-app/data/tinycollege/staff.adm b/asterixdb/asterix-app/data/tinycollege/staff.adm
index 4bfdd4b..d24bd87 100644
--- a/asterixdb/asterix-app/data/tinycollege/staff.adm
+++ b/asterixdb/asterix-app/data/tinycollege/staff.adm
@@ -1,7 +1,8 @@
-{ "id": 14, "name": "Alex", "office": "A", "employment": interval(date("2003-01-01"), date("2008-01-01")) }
-{ "id": 13, "name": "Elisabeth", "office": "B", "employment": interval(date("2002-01-01"), date("2010-01-01")) }
-{ "id": 16, "name": "Franklin", "office": "A", "employment": interval(date("2004-01-01"), date("2009-01-01")) }
-{ "id": 15, "name": "Henry", "office": "C", "employment": interval(date("2003-01-01"), date("2008-01-01")) }
-{ "id": 17, "name": "Maryann", "office": "B", "employment": interval(date("2006-01-01"), date("2010-01-01")) }
-{ "id": 11, "name": "Vicky", "office": "D", "employment": interval(date("2001-01-01"), date("2010-01-01")) }
-{ "id": 12, "name": "Zack", "office": "A", "employment": interval(date("2002-01-01"), date("2003-01-01")) }
+{ "id": 14, "name": "Alex", "office": "A", "employment": interval(date("2003-01-01"), date("2008-01-01")), "vacation": interval(datetime("2003-01-01T00:00:00.0"), datetime("2008-01-01T00:00:00.0")), "office_hours": interval(time("03:00:00.0+00:00"), time("08:00:00.0+00:00"))}
+{ "id": 13, "name": "Elisabeth", "office": "B", "employment": interval(date("2002-01-01"), date("2010-01-01")), "vacation": interval(datetime("2002-01-01T00:00:00.0"), datetime("2010-01-01T00:00:00.0")), "office_hours": interval(time("02:00:00.0+00:00"), time("10:00:00.0+00:00"))}
+{ "id": 16, "name": "Franklin", "office": "A", "employment": interval(date("2004-01-01"), date("2009-01-01")), "vacation": interval(datetime("2004-01-01T00:00:00.0"), datetime("2009-01-01T00:00:00.0")), "office_hours": interval(time("04:00:00.0+00:00"), time("09:00:00.0+00:00"))}
+{ "id": 15, "name": "Henry", "office": "C", "employment": interval(date("2003-01-01"), date("2008-01-01")), "vacation": interval(datetime("2003-01-01T00:00:00.0"), datetime("2008-01-01T00:00:00.0")), "office_hours": interval(time("03:00:00.0+00:00"), time("08:00:00.0+00:00"))}
+{ "id": 17, "name": "MaryAnn", "office": "B", "employment": interval(date("2006-01-01"), date("2010-01-01")), "vacation": interval(datetime("2006-01-01T00:00:00.0"), datetime("2010-01-01T00:00:00.0")), "office_hours": interval(time("06:00:00.0+00:00"), time("10:00:00.0+00:00"))}
+{ "id": 11, "name": "Vicky", "office": "D", "employment": interval(date("2001-01-01"), date("2010-01-01")), "vacation": interval(datetime("2001-01-01T00:00:00.0"), datetime("2010-01-01T00:00:00.0")), "office_hours": interval(time("01:00:00.0+00:00"), time("10:00:00.0+00:00"))}
+{ "id": 12, "name": "Zack", "office": "A", "employment": interval(date("2002-01-01"), date("2003-01-01")), "vacation": interval(datetime("2002-01-01T00:00:00.0"), datetime("2003-01-01T00:00:00.0")), "office_hours": interval(time("02:00:00.0+00:00"), time("03:00:00.0+00:00"))}
+{ "id": 18, "name": "Jake", "office": "C", "employment": interval(date("2006-01-01"), date("2008-01-01")), "vacation": interval(datetime("2006-01-01T00:00:00.0"), datetime("2008-01-01T00:00:00.0")), "office_hours": interval(time("06:00:00.0+00:00"), time("08:00:00.0+00:00"))}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tinycollege/students.adm b/asterixdb/asterix-app/data/tinycollege/students.adm
index b869075..d9447d2 100644
--- a/asterixdb/asterix-app/data/tinycollege/students.adm
+++ b/asterixdb/asterix-app/data/tinycollege/students.adm
@@ -1,7 +1,7 @@
-{ "id": 22, "name": "Charles", "office": "X", "attendance": interval(date("2001-01-01"), date("2004-01-01")) }
-{ "id": 23, "name": "Frank", "office": "Y", "attendance": interval(date("2001-01-01"), date("2004-01-01")) }
-{ "id": 25, "name": "Karen", "office": "Y", "attendance": interval(date("2007-01-01"), date("2009-01-01")) }
-{ "id": 24, "name": "Mary", "office": "Y", "attendance": interval(date("2002-01-01"), date("2005-01-01")) }
-{ "id": 21, "name": "Olga", "office": "Z", "attendance": interval(date("2001-01-01"), date("2003-01-01")) }
-{ "id": 26, "name": "Steve", "office": "Z", "attendance": interval(date("2007-01-01"), date("2010-01-01")) }
-{ "id": 27, "name": "Tess", "office": "X", "attendance": interval(date("2008-01-01"), date("2010-01-01")) }
+{ "id": 22, "name": "Charles", "office": "X", "attendance": interval(date("2001-01-01"), date("2004-01-01")), "break": interval(datetime("2001-01-01T00:00:00.0"), datetime("2004-01-01T00:00:00.0")), "work_hours": interval(time("01:00:00.0+00:00"), time("04:00:00.0+00:00"))}
+{ "id": 23, "name": "Frank", "office": "Y", "attendance": interval(date("2001-01-01"), date("2004-01-01")), "break": interval(datetime("2001-01-01T00:00:00.0"), datetime("2004-01-01T00:00:00.0")), "work_hours": interval(time("01:00:00.0+00:00"), time("04:00:00.0+00:00"))}
+{ "id": 25, "name": "Karen", "office": "Y", "attendance": interval(date("2007-01-01"), date("2009-01-01")), "break": interval(datetime("2007-01-01T00:00:00.0"), datetime("2009-01-01T00:00:00.0")), "work_hours": interval(time("07:00:00.0+00:00"), time("09:00:00.0+00:00"))}
+{ "id": 24, "name": "Mary", "office": "Y", "attendance": interval(date("2002-01-01"), date("2005-01-01")), "break": interval(datetime("2002-01-01T00:00:00.0"), datetime("2005-01-01T00:00:00.0")), "work_hours": interval(time("02:00:00.0+00:00"), time("05:00:00.0+00:00"))}
+{ "id": 21, "name": "Olga", "office": "Z", "attendance": interval(date("2001-01-01"), date("2003-01-01")), "break": interval(datetime("2001-01-01T00:00:00.0"), datetime("2003-01-01T00:00:00.0")), "work_hours": interval(time("01:00:00.0+00:00"), time("03:00:00.0+00:00"))}
+{ "id": 26, "name": "Steve", "office": "Z", "attendance": interval(date("2007-01-01"), date("2010-01-01")), "break": interval(datetime("2007-01-01T00:00:00.0"), datetime("2010-01-01T00:00:00.0")), "work_hours": interval(time("07:00:00.0+00:00"), time("10:00:00.0+00:00"))}
+{ "id": 27, "name": "Tess", "office": "X", "attendance": interval(date("2008-01-01"), date("2010-01-01")), "break": interval(datetime("2008-01-01T00:00:00.0"), datetime("2010-01-01T00:00:00.0")), "work_hours": interval(time("08:00:00.0+00:00"), time("10:00:00.0+00:00"))}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
index a33e6dd..bd35c4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval after
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff": f.employment, "student" : d.attendance }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-after`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_after(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
index e8ebb86..acea862 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval before
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -45,4 +45,6 @@
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-before`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_before(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
index 854cd59..bc4ec80 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval covered_by
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covered-by`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covered_by(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
index 1fa198d..a054f13 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval covers
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covers`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covers(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
index 1d48700..d683999 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval ended_by
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in ended_by because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ended-by`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ended_by(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
index 14346f6..946edbc 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval ends
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in ends because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ends`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ends(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
index 7bce341..9dc0710 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval meets
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in meets because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-meets`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_meets(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
index 19f206d..720daf1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval met_by
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in met_by because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-met-by`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_met_by(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
index 72a1f50..38a5149 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval overlapped_by
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapped-by`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapped_by(f.employment, d.attendance);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
index f1c3a68..71b43e7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval overlapping
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapping`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapping(f.employment, d.attendance);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
index efb224f..2bf3fd4 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
@@ -17,9 +17,9 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval overlaps
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +43,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlaps`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlaps(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
index 410504d..b19044a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval started_by
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in started_by because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-started-by`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_started_by(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
index 87ec386..38d855d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
@@ -17,9 +17,11 @@
* under the License.
*/
/*
- * Description : Check temporal join functionality for interval
+ * Description : Check temporal join functionality for interval starts
* Expected Result : Success
- * Date : 21th Oct, 2019
+ * Date : 10th Aug, 2019
+ * Note : Range hint is not used in ended_by because it is not implemented.
+ * This test will default to Hybrid Hash Join
*/
drop dataverse TinyCollege if exists;
@@ -43,6 +45,8 @@
create dataset Students(StudentType)
primary key name;
-select element { "staff" : f.name, "student" : d.name }
+select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-starts`(f.employment, d.attendance);
\ No newline at end of file
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_starts(f.employment, d.attendance);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
index 6ac3ccc..1aab448 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_after/interval_after.3.query.plan
@@ -2,21 +2,30 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
- -- ONE_TO_ONE_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|
- -- BROADCAST_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|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$34(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
index 757511d..883bfed 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_before/interval_before.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE [$$34(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
index 757511d..a9f6d74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$34(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/temporal/interval_joins/interval_covers/interval_covers.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
index 757511d..d67dc29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_covers/interval_covers.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$34,$$35,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
index 757511d..a9f6d74 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$34(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
index 757511d..9f4a00d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$34,$$35,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$36,$$37,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
index 757511d..d67dc29 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.plan
@@ -4,18 +4,28 @@
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- NESTED_LOOP |PARTITIONED|
+ -- INTERVAL_MERGE_JOIN [$$30] [$$31] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |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|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- STABLE_SORT [$$30(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE [{$$34,$$35,ASC}] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |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|
+ -- STABLE_SORT [$$31(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- RANGE_PARTITION_EXCHANGE [$$36(ASC)] RANGE_MAP:{SPLIT:3} |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.1.ddl.sqlpp
index fdef960..9ea732a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,10 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.10.query.sqlpp
new file mode 100644
index 0000000..3504834
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_after(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.11.query.sqlpp
new file mode 100644
index 0000000..45406b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_after(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.12.query.sqlpp
new file mode 100644
index 0000000..3baabf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_after(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.13.query.sqlpp
new file mode 100644
index 0000000..6b91ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_after(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.14.query.sqlpp
new file mode 100644
index 0000000..c5f2280
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_after(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
index a303827..6668205 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 9th January 2019
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-after`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_after(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.4.query.sqlpp
index fa439c1..31ba5aa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-after`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_after(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.5.query.sqlpp
index 39e3372..c9308fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-after`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_after(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.6.query.sqlpp
index eb2797b..48118d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-after`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_after(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.7.query.sqlpp
new file mode 100644
index 0000000..a013ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_after(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.8.query.sqlpp
new file mode 100644
index 0000000..690e5b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_after(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.9.query.sqlpp
new file mode 100644
index 0000000..c57cfc7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_after/interval_after.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_after(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.10.query.sqlpp
new file mode 100644
index 0000000..9e22e04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_before(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.11.query.sqlpp
new file mode 100644
index 0000000..b93f969
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_before(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.12.query.sqlpp
new file mode 100644
index 0000000..6ffb1cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_before(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.13.query.sqlpp
new file mode 100644
index 0000000..295f895
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_before(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.14.query.sqlpp
new file mode 100644
index 0000000..6d0e028
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_before(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
index 4644367..faf9e7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-before`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_before(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.4.query.sqlpp
index 8db7596..6c1f734 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-before`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_before(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.5.query.sqlpp
index 3769ff3..c7e34d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-before`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_before(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.6.query.sqlpp
index 70ab1ea..27185d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-before`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_before(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.7.query.sqlpp
new file mode 100644
index 0000000..c184704
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_before(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.8.query.sqlpp
new file mode 100644
index 0000000..7425bce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_before(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.9.query.sqlpp
new file mode 100644
index 0000000..5089141
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_before/interval_before.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_before(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.10.query.sqlpp
new file mode 100644
index 0000000..d71029a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covered_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.11.query.sqlpp
new file mode 100644
index 0000000..e635228
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covered_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.12.query.sqlpp
new file mode 100644
index 0000000..25d22d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covered_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.13.query.sqlpp
new file mode 100644
index 0000000..1f287ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covered_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.14.query.sqlpp
new file mode 100644
index 0000000..881f62e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covered_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
index cdd6744..3458df9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covered-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covered_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.4.query.sqlpp
index 2209a6a..2deb6e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covered-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covered_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.5.query.sqlpp
index 69ac8d5..fa931af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-covered-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covered_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.6.query.sqlpp
index 467200a..0010b26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-covered-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covered_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.7.query.sqlpp
new file mode 100644
index 0000000..5bf5d4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covered_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.8.query.sqlpp
new file mode 100644
index 0000000..83f7beb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covered_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.9.query.sqlpp
new file mode 100644
index 0000000..46ba991
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covered_by/interval_covered_by.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covered_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.10.query.sqlpp
new file mode 100644
index 0000000..16f53dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covers(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.11.query.sqlpp
new file mode 100644
index 0000000..94db651
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covers(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.12.query.sqlpp
new file mode 100644
index 0000000..fbc64ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covers(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.13.query.sqlpp
new file mode 100644
index 0000000..bdd439e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covers(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.14.query.sqlpp
new file mode 100644
index 0000000..dddf7bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_covers(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
index 2816cfb..bc2e0b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covers`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covers(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.4.query.sqlpp
index 0241040..c22cc60 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-covers`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covers(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.5.query.sqlpp
index 8e600d0..9a2c572 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-covers`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covers(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.6.query.sqlpp
index d4a3760..89d9ada 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-covers`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_covers(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.7.query.sqlpp
new file mode 100644
index 0000000..ba156f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covers(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.8.query.sqlpp
new file mode 100644
index 0000000..051620d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covers(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.9.query.sqlpp
new file mode 100644
index 0000000..dbd1fba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_covers/interval_covers.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_covers(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.1.ddl.sqlpp
index fdef960..07166ca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.10.query.sqlpp
new file mode 100644
index 0000000..390daaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ended_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.11.query.sqlpp
new file mode 100644
index 0000000..2986aa9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ended_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.12.query.sqlpp
new file mode 100644
index 0000000..0520104
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ended_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.13.query.sqlpp
new file mode 100644
index 0000000..62f72b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ended_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.14.query.sqlpp
new file mode 100644
index 0000000..09ff8b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ended_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.2.update.sqlpp
index 4334476..49648a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
index 96b63c5..5d410ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ended-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ended_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.4.query.sqlpp
index 74929b2..ef30f3e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ended-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ended_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.5.query.sqlpp
index 08f33a5..f4aae13 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-ended-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ended_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.6.query.sqlpp
index bb4e8b2..bd4dc8d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-ended-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ended_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.7.query.sqlpp
new file mode 100644
index 0000000..5d0e86d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ended_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.8.query.sqlpp
new file mode 100644
index 0000000..053535b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ended_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.9.query.sqlpp
new file mode 100644
index 0000000..c7d21f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ended_by/interval_ended_by.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ended_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ended_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.1.ddl.sqlpp
index fdef960..e3c2e76 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.10.query.sqlpp
new file mode 100644
index 0000000..80f1641
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ends(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.11.query.sqlpp
new file mode 100644
index 0000000..a12b527
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ends(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.12.query.sqlpp
new file mode 100644
index 0000000..fd758e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ends(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.13.query.sqlpp
new file mode 100644
index 0000000..0f8a057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ends(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.14.query.sqlpp
new file mode 100644
index 0000000..010c04e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_ends(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.2.update.sqlpp
index 4334476..881d227 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
index 50ae865..42e9380 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ends`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ends(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.4.query.sqlpp
index 86d9ff4..7ad2449 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-ends`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ends(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.5.query.sqlpp
index cf316a3..6751fc1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-ends`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ends(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.6.query.sqlpp
index c9faeda..61856be 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-ends`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_ends(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.7.query.sqlpp
new file mode 100644
index 0000000..7b05616
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ends(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.8.query.sqlpp
new file mode 100644
index 0000000..a0d6a58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ends(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.9.query.sqlpp
new file mode 100644
index 0000000..e1b5aa0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_ends/interval_ends.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval ends because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_ends(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.1.ddl.sqlpp
index fdef960..107e651 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.10.query.sqlpp
new file mode 100644
index 0000000..2998ed3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_meets(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.11.query.sqlpp
new file mode 100644
index 0000000..31d89ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_meets(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.12.query.sqlpp
new file mode 100644
index 0000000..b79583f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_meets(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.13.query.sqlpp
new file mode 100644
index 0000000..1f1083b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_meets(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.14.query.sqlpp
new file mode 100644
index 0000000..8d8ad81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_meets(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.2.update.sqlpp
index 4334476..95937ec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
index 244cd62..80dd09c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-meets`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_meets(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.4.query.sqlpp
index ac9dd33..24dc50f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-meets`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_meets(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.5.query.sqlpp
index 8fd6a60..5c068942 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-meets`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_meets(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.6.query.sqlpp
index 34c9a12..f438e6a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-meets`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_meets(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.7.query.sqlpp
new file mode 100644
index 0000000..b2723f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_meets(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.8.query.sqlpp
new file mode 100644
index 0000000..6a99fbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_meets(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.9.query.sqlpp
new file mode 100644
index 0000000..53f6a41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_meets/interval_meets.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval meets because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_meets(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.1.ddl.sqlpp
index fdef960..2d76f34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.10.query.sqlpp
new file mode 100644
index 0000000..9b0113a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_met_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.11.query.sqlpp
new file mode 100644
index 0000000..d2aa20b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_met_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.12.query.sqlpp
new file mode 100644
index 0000000..c8156dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_met_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.13.query.sqlpp
new file mode 100644
index 0000000..2d18ea3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_met_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.14.query.sqlpp
new file mode 100644
index 0000000..7105983
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_met_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.2.update.sqlpp
index 4334476..c260fec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
index f1f39a9..0aaf497 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-met-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_met_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.4.query.sqlpp
index fa2e9ed..57aa56f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-met-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_met_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.5.query.sqlpp
index 2755407..1bff1e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-met-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_met_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.6.query.sqlpp
index 319cb32..a601324 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-met-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_met_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.7.query.sqlpp
new file mode 100644
index 0000000..fdf140c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_met_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.8.query.sqlpp
new file mode 100644
index 0000000..9929fe8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_met_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.9.query.sqlpp
new file mode 100644
index 0000000..5a0a9da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_met_by/interval_met_by.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval met_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_met_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.query.sqlpp
new file mode 100644
index 0000000..75bd6de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapped_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.query.sqlpp
new file mode 100644
index 0000000..0d78bf9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapped_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.query.sqlpp
new file mode 100644
index 0000000..4b0761b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapped_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.query.sqlpp
new file mode 100644
index 0000000..e8bfd5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapped_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.query.sqlpp
new file mode 100644
index 0000000..6e95625
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapped_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
index 2fc8575..9033c40 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapped-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapped_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.query.sqlpp
index 1f80cd0..5389119 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapped-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapped_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.5.query.sqlpp
index e32af46..1ff426f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlapped-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapped_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.query.sqlpp
index c8b77f3..422ce76 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlapped-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapped_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.query.sqlpp
new file mode 100644
index 0000000..89f8a60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapped_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.query.sqlpp
new file mode 100644
index 0000000..02845b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapped_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.query.sqlpp
new file mode 100644
index 0000000..08d2c04
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapped_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.sqlpp
new file mode 100644
index 0000000..e1ce7ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapping(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.sqlpp
new file mode 100644
index 0000000..e30bf20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapping(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.sqlpp
new file mode 100644
index 0000000..a994ebf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapping(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.sqlpp
new file mode 100644
index 0000000..0e30efb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapping(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.sqlpp
new file mode 100644
index 0000000..bcd2606
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlapping(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
index 91589f2..ddcae39 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapping`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapping(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.4.query.sqlpp
index 12b5daf..e7f62d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlapping`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapping(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.sqlpp
index b0f4423..eaba9a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlapping`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapping(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.sqlpp
index 94a690e..5e601e0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlapping`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlapping(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.sqlpp
new file mode 100644
index 0000000..ba64ef0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapping(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.8.query.sqlpp
new file mode 100644
index 0000000..3656661
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapping(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.sqlpp
new file mode 100644
index 0000000..84b6252
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlapping(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.sqlpp
index fdef960..ae840c3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +29,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +40,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.sqlpp
new file mode 100644
index 0000000..bd7fc86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlaps(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.sqlpp
new file mode 100644
index 0000000..dc51559
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlaps(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.sqlpp
new file mode 100644
index 0000000..50e1fbb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlaps(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.sqlpp
new file mode 100644
index 0000000..bd750f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlaps(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.sqlpp
new file mode 100644
index 0000000..91f42a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_overlaps(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.sqlpp
index 4334476..b7cd541 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.sqlpp
@@ -19,7 +19,7 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
index 579eda5..717f7e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlaps`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlaps(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.sqlpp
index 1327d67..bdb8ec4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-overlaps`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlaps(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.sqlpp
index ac40622..f5bbac4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlaps`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlaps(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.sqlpp
index 65f43c5..03d8da5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.sqlpp
@@ -19,12 +19,14 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-overlaps`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_overlaps(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.sqlpp
new file mode 100644
index 0000000..c4ff119
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlaps(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.sqlpp
new file mode 100644
index 0000000..8e4eca8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlaps(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.sqlpp
new file mode 100644
index 0000000..4de8f1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_overlaps(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.1.ddl.sqlpp
index fdef960..5fcd96e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.10.query.sqlpp
new file mode 100644
index 0000000..319efac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_started_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.11.query.sqlpp
new file mode 100644
index 0000000..d39164d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_started_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.12.query.sqlpp
new file mode 100644
index 0000000..70efc7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_started_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.13.query.sqlpp
new file mode 100644
index 0000000..c6b20ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_started_by(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.14.query.sqlpp
new file mode 100644
index 0000000..76dd651
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_started_by(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.2.update.sqlpp
index 4334476..5fbdf7d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
index ca93d0e..f3f805d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-started-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_started_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.4.query.sqlpp
index 5c5ae25..af5b059 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-started-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_started_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.5.query.sqlpp
index 91436cc..e414153 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-started-by`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_started_by(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.6.query.sqlpp
index d3ea093..3867b87 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-started-by`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_started_by(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.7.query.sqlpp
new file mode 100644
index 0000000..2b3a26f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_started_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.8.query.sqlpp
new file mode 100644
index 0000000..b4de8e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_started_by(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.9.query.sqlpp
new file mode 100644
index 0000000..ab02067
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_started_by/interval_started_by.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval started_by because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_started_by(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.1.ddl.sqlpp
index fdef960..3326abb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.1.ddl.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
drop dataverse TinyCollege if exists;
@@ -29,7 +31,9 @@
create type StaffType as open {
name: string,
office: string,
- employment: interval
+ employment: interval,
+ vacation: interval,
+ office_hours: interval
};
create dataset Staff(StaffType)
primary key name;
@@ -38,7 +42,9 @@
create type StudentType as open {
name: string,
office: string,
- attendance: interval
+ attendance: interval,
+ break: interval,
+ work_hours: interval
};
create dataset Students(StudentType)
primary key name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.10.query.sqlpp
new file mode 100644
index 0000000..71a89a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_starts(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.11.query.sqlpp
new file mode 100644
index 0000000..4e567c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.11.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_starts(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.12.query.sqlpp
new file mode 100644
index 0000000..cd7cf4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.12.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_starts(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.13.query.sqlpp
new file mode 100644
index 0000000..8146936
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.13.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_starts(f.office_hours, d.work_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.14.query.sqlpp
new file mode 100644
index 0000000..1fea371
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.14.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [time("03:30:00.0+00:00"), time("05:59:00.0+00:00"), time("08:30:00.0+00:00")] */
+interval_starts(d.work_hours, f.office_hours)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.2.update.sqlpp
index 4334476..552dbc7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.2.update.sqlpp
@@ -19,7 +19,9 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
index 915a717..51788b7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.3.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-starts`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_starts(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.4.query.sqlpp
index 2e8b27e..cf9d660 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.4.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Staff as f, Students as d
-where `interval-starts`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_starts(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.5.query.sqlpp
index f5aa575..fe13a01 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.5.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-starts`(f.employment, d.attendance)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_starts(f.employment, d.attendance)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.6.query.sqlpp
index 146ecd1..4621f50 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.6.query.sqlpp
@@ -19,12 +19,16 @@
/*
* Description : Check temporal join functionality for interval
* Expected Result : Success
- * Date : 26th Jun, 2015
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
*/
use TinyCollege;
select element { "staff" : f.name, "student" : d.name }
from Students as d, Staff as f
-where `interval-starts`(d.attendance, f.employment)
+where
+/*+ range [date("2003-06-30"), date("2005-12-31"), date("2008-06-30")] */
+interval_starts(d.attendance, f.employment)
order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.7.query.sqlpp
new file mode 100644
index 0000000..49490f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.7.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_starts(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.8.query.sqlpp
new file mode 100644
index 0000000..5430332
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.8.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Staff as f, Students as d
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_starts(d.break, f.vacation)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.9.query.sqlpp
new file mode 100644
index 0000000..e97dafa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_joins/interval_starts/interval_starts.9.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Check temporal join functionality for interval
+ * Expected Result : Success
+ * Date : 10th Aug, 2020
+ * Note : Range hint is not picked for interval starts because it is not implemented
+ * : It will default to Hybrid Has Join.
+ */
+
+use TinyCollege;
+
+select element { "staff" : f.name, "student" : d.name }
+from Students as d, Staff as f
+where
+/*+ range [datetime("2003-06-30T00:00:00.0"), datetime("2005-12-31T00:00:00.0"), datetime("2008-06-30T00:00:00.0")] */
+interval_starts(f.vacation, d.break)
+order by f.name, d.name;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.10.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.10.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.11.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.11.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.12.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.12.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.13.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.13.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.14.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.14.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.3.adm
index f38e40b..93c0eb5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.3.adm
@@ -1,5 +1,9 @@
{ "staff": "Franklin", "student": "Olga" }
-{ "staff": "Maryann", "student": "Charles" }
-{ "staff": "Maryann", "student": "Frank" }
-{ "staff": "Maryann", "student": "Mary" }
-{ "staff": "Maryann", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.5.adm
index f38e40b..93c0eb5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.5.adm
@@ -1,5 +1,9 @@
{ "staff": "Franklin", "student": "Olga" }
-{ "staff": "Maryann", "student": "Charles" }
-{ "staff": "Maryann", "student": "Frank" }
-{ "staff": "Maryann", "student": "Mary" }
-{ "staff": "Maryann", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.7.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.7.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.8.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.8.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.9.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_after/interval_after.9.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.10.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.10.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.11.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.11.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.12.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.12.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.13.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.13.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.14.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.14.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.3.adm
index 509693a..c15304d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.3.adm
@@ -1,3 +1,3 @@
{ "staff": "Zack", "student": "Karen" }
{ "staff": "Zack", "student": "Steve" }
-{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.4.adm
index f38e40b..93c0eb5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.4.adm
@@ -1,5 +1,9 @@
{ "staff": "Franklin", "student": "Olga" }
-{ "staff": "Maryann", "student": "Charles" }
-{ "staff": "Maryann", "student": "Frank" }
-{ "staff": "Maryann", "student": "Mary" }
-{ "staff": "Maryann", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.5.adm
index 509693a..c15304d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.5.adm
@@ -1,3 +1,3 @@
{ "staff": "Zack", "student": "Karen" }
{ "staff": "Zack", "student": "Steve" }
-{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.6.adm
index f38e40b..93c0eb5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.6.adm
@@ -1,5 +1,9 @@
{ "staff": "Franklin", "student": "Olga" }
-{ "staff": "Maryann", "student": "Charles" }
-{ "staff": "Maryann", "student": "Frank" }
-{ "staff": "Maryann", "student": "Mary" }
-{ "staff": "Maryann", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.7.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.7.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.8.adm
new file mode 100644
index 0000000..93c0eb5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.8.adm
@@ -0,0 +1,9 @@
+{ "staff": "Franklin", "student": "Olga" }
+{ "staff": "Jake", "student": "Charles" }
+{ "staff": "Jake", "student": "Frank" }
+{ "staff": "Jake", "student": "Mary" }
+{ "staff": "Jake", "student": "Olga" }
+{ "staff": "MaryAnn", "student": "Charles" }
+{ "staff": "MaryAnn", "student": "Frank" }
+{ "staff": "MaryAnn", "student": "Mary" }
+{ "staff": "MaryAnn", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.9.adm
new file mode 100644
index 0000000..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.9.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.10.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.10.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.11.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.11.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.12.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.12.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.13.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.13.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.14.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.14.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.4.adm
index 4e22101..90e39d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.4.adm
@@ -3,9 +3,9 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.6.adm
index 4e22101..90e39d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.6.adm
@@ -3,9 +3,9 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.7.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.7.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.8.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.8.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.9.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covered_by/interval_covered_by.9.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.10.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.10.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.11.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.11.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.12.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.12.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.13.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.13.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.14.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.14.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.3.adm
index 4e22101..90e39d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.3.adm
@@ -3,9 +3,9 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.adm
index 4e22101..90e39d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.adm
@@ -3,9 +3,9 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.7.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.7.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.8.adm
new file mode 100644
index 0000000..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.8.adm
@@ -0,0 +1,4 @@
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.9.adm
new file mode 100644
index 0000000..90e39d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.9.adm
@@ -0,0 +1,15 @@
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.10.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.10.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.11.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.11.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.12.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.12.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.13.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.13.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.14.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.14.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.4.adm
index 5af9101..1fc19e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.4.adm
@@ -1,7 +1,7 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Steve" }
{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.6.adm
index 5af9101..1fc19e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.6.adm
@@ -1,7 +1,7 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Steve" }
{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.7.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.7.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.8.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.8.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.9.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ended_by/interval_ended_by.9.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.10.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.10.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.11.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.11.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.12.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.12.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.13.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.13.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.14.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.14.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.3.adm
index 5af9101..1fc19e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.3.adm
@@ -1,7 +1,7 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Steve" }
{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.5.adm
index 5af9101..1fc19e7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.5.adm
@@ -1,7 +1,7 @@
{ "staff": "Elisabeth", "student": "Steve" }
{ "staff": "Elisabeth", "student": "Tess" }
{ "staff": "Franklin", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Steve" }
{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.7.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.7.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.8.adm
new file mode 100644
index 0000000..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.8.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.9.adm
new file mode 100644
index 0000000..1fc19e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.9.adm
@@ -0,0 +1,7 @@
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.10.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.10.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.11.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.11.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.12.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.12.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.13.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.13.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.14.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.14.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.3.adm
index 5568195..8cafa72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.3.adm
@@ -1,2 +1,3 @@
{ "staff": "Alex", "student": "Tess" }
{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.5.adm
index 5568195..8cafa72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.5.adm
@@ -1,2 +1,3 @@
{ "staff": "Alex", "student": "Tess" }
{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.7.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.7.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.8.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.8.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.9.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.9.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.10.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.10.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.11.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.11.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.12.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.12.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.13.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.13.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.14.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.14.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.4.adm
index 5568195..8cafa72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.4.adm
@@ -1,2 +1,3 @@
{ "staff": "Alex", "student": "Tess" }
{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.6.adm
index 5568195..8cafa72 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.6.adm
@@ -1,2 +1,3 @@
{ "staff": "Alex", "student": "Tess" }
{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.7.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.7.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.8.adm
new file mode 100644
index 0000000..8cafa72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.8.adm
@@ -0,0 +1,3 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
+{ "staff": "Jake", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.9.adm
new file mode 100644
index 0000000..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_met_by/interval_met_by.9.adm
@@ -0,0 +1,4 @@
+{ "staff": "Alex", "student": "Olga" }
+{ "staff": "Franklin", "student": "Charles" }
+{ "staff": "Franklin", "student": "Frank" }
+{ "staff": "Henry", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.10.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.11.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.12.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.13.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.14.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.adm
index 94ac56a..7ab4d4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.4.adm
@@ -4,3 +4,5 @@
{ "staff": "Franklin", "student": "Tess" }
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.adm
index 94ac56a..7ab4d4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.6.adm
@@ -4,3 +4,5 @@
{ "staff": "Franklin", "student": "Tess" }
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.7.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.8.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapped_by/interval_overlapped_by.9.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.10.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.10.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.11.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.11.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.12.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.12.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.13.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.13.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.14.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.14.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.adm
index 4ecd143..dc01c06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.3.adm
@@ -19,9 +19,11 @@
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Mary" }
{ "staff": "Henry", "student": "Steve" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.4.adm
index 4ecd143..dc01c06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.4.adm
@@ -19,9 +19,11 @@
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Mary" }
{ "staff": "Henry", "student": "Steve" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm
index 4ecd143..dc01c06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm
@@ -19,9 +19,11 @@
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Mary" }
{ "staff": "Henry", "student": "Steve" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm
index 4ecd143..dc01c06 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm
@@ -19,9 +19,11 @@
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Mary" }
{ "staff": "Henry", "student": "Steve" }
-{ "staff": "Maryann", "student": "Karen" }
-{ "staff": "Maryann", "student": "Steve" }
-{ "staff": "Maryann", "student": "Tess" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
{ "staff": "Vicky", "student": "Charles" }
{ "staff": "Vicky", "student": "Frank" }
{ "staff": "Vicky", "student": "Karen" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.7.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.7.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.8.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.8.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.9.adm
new file mode 100644
index 0000000..dc01c06
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.9.adm
@@ -0,0 +1,37 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Karen" }
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Elisabeth", "student": "Steve" }
+{ "staff": "Elisabeth", "student": "Tess" }
+{ "staff": "Franklin", "student": "Karen" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Mary" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Karen" }
+{ "staff": "MaryAnn", "student": "Steve" }
+{ "staff": "MaryAnn", "student": "Tess" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Karen" }
+{ "staff": "Vicky", "student": "Mary" }
+{ "staff": "Vicky", "student": "Olga" }
+{ "staff": "Vicky", "student": "Steve" }
+{ "staff": "Vicky", "student": "Tess" }
+{ "staff": "Zack", "student": "Charles" }
+{ "staff": "Zack", "student": "Frank" }
+{ "staff": "Zack", "student": "Mary" }
+{ "staff": "Zack", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.10.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.10.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.11.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.11.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.12.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.12.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.13.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.13.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.14.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.14.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
index 94ac56a..7ab4d4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
@@ -4,3 +4,5 @@
{ "staff": "Franklin", "student": "Tess" }
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
index 94ac56a..7ab4d4f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
@@ -4,3 +4,5 @@
{ "staff": "Franklin", "student": "Tess" }
{ "staff": "Henry", "student": "Karen" }
{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm
new file mode 100644
index 0000000..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm
@@ -0,0 +1,10 @@
+{ "staff": "Alex", "student": "Charles" }
+{ "staff": "Alex", "student": "Frank" }
+{ "staff": "Alex", "student": "Mary" }
+{ "staff": "Elisabeth", "student": "Charles" }
+{ "staff": "Elisabeth", "student": "Frank" }
+{ "staff": "Elisabeth", "student": "Olga" }
+{ "staff": "Franklin", "student": "Mary" }
+{ "staff": "Henry", "student": "Charles" }
+{ "staff": "Henry", "student": "Frank" }
+{ "staff": "Henry", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.9.adm
new file mode 100644
index 0000000..7ab4d4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.9.adm
@@ -0,0 +1,8 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
+{ "staff": "Jake", "student": "Karen" }
+{ "staff": "Jake", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.10.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.10.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.11.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.11.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.12.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.12.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.13.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.13.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.14.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.14.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.7.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.7.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.8.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.8.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.9.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_started_by/interval_started_by.9.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.10.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.10.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.11.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.11.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.12.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.12.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.13.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.13.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.14.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.14.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.14.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.7.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.7.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.8.adm
new file mode 100644
index 0000000..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.8.adm
@@ -0,0 +1,4 @@
+{ "staff": "Elisabeth", "student": "Mary" }
+{ "staff": "Vicky", "student": "Charles" }
+{ "staff": "Vicky", "student": "Frank" }
+{ "staff": "Vicky", "student": "Olga" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.9.adm
new file mode 100644
index 0000000..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.9.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Mary" }
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 19f5614..58af775 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -13812,7 +13812,7 @@
<expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "skip-index", "hash-bcast" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "skip-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "skip-index", "range" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
</compilation-unit>
</test-case>
@@ -13835,7 +13835,7 @@
<expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "indexnl", "skip-index", "hash-bcast" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index" expected at this location</expected-warn>
- <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "skip-index" expected at this location</expected-warn>
+ <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "skip-index", "range" expected at this location</expected-warn>
<expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
</compilation-unit>
</test-case>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RangeAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RangeAnnotation.java
new file mode 100644
index 0000000..4465167
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/RangeAnnotation.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.asterix.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class RangeAnnotation implements IExpressionAnnotation {
+
+ private RangeMap map;
+
+ @Override
+ public Object getObject() {
+ return map;
+ }
+
+ @Override
+ public void setObject(Object side) {
+ this.map = (RangeMap) side;
+ }
+
+ @Override
+ public IExpressionAnnotation copy() {
+ RangeAnnotation rangAnn = new RangeAnnotation();
+ rangAnn.map = map;
+ return rangAnn;
+ }
+
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index d2c851f..e23ea59 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -59,6 +59,7 @@
import org.apache.asterix.common.annotations.InsertRandIntDataGen;
import org.apache.asterix.common.annotations.ListDataGen;
import org.apache.asterix.common.annotations.ListValFileDataGen;
+import org.apache.asterix.common.annotations.RangeAnnotation;
import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
import org.apache.asterix.common.annotations.TypeDataGen;
import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
@@ -263,7 +264,7 @@
public DataverseName dataverse;
public String library;
public String function;
- public SqlppHint hint;
+ public Token hintToken;
public SourceLocation sourceLoc;
}
@@ -2286,19 +2287,19 @@
FunctionName FunctionName() throws ParseException:
{
- Triple<List<String>, SourceLocation, SqlppHint> prefix = null;
+ Triple<List<String>, SourceLocation, Token> prefix = null;
String suffix = null;
}
{
// Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
// that copy must be kept in sync with this code
prefix = MultipartIdentifierWithHints(SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT,
- SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT)
+ SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.RANGE_HINT)
(<SHARP> suffix = Identifier())?
{
FunctionName result = new FunctionName();
result.sourceLoc = prefix.second;
- result.hint = prefix.third;
+ result.hintToken = prefix.third;
List<String> list = prefix.first;
int ln = list.size();
String last = list.get(ln - 1);
@@ -2442,7 +2443,7 @@
List<String> MultipartIdentifier() throws ParseException:
{
- Triple<List<String>, SourceLocation, SqlppHint> result = null;
+ Triple<List<String>, SourceLocation, Token> result = null;
}
{
result = MultipartIdentifierWithHints(null)
@@ -2451,12 +2452,12 @@
}
}
-Triple<List<String>, SourceLocation, SqlppHint> MultipartIdentifierWithHints(SqlppHint... expectedHints)
+Triple<List<String>, SourceLocation, Token> MultipartIdentifierWithHints(SqlppHint... expectedHints)
throws ParseException:
{
List<String> list = new ArrayList<String>();
SourceLocation sourceLoc = null;
- SqlppHint hint = null;
+ Token hint = null;
String item = null;
}
{
@@ -2465,15 +2466,12 @@
list.add(item);
sourceLoc = getSourceLocation(token);
if (expectedHints != null && expectedHints.length > 0) {
- Token hintToken = fetchHint(token, expectedHints);
- if (hintToken != null) {
- hint = hintToken.hint;
- }
+ hint = fetchHint(token, expectedHints);
}
}
(<DOT> item = Identifier() { list.add(item); } )*
{
- return new Triple<List<String>, SourceLocation, SqlppHint>(list, sourceLoc, hint);
+ return new Triple<List<String>, SourceLocation, Token>(list, sourceLoc, hint);
}
}
@@ -3442,14 +3440,27 @@
return windowExpr;
} else {
CallExpr callExpr = new CallExpr(signature, argList, filterExpr);
- if (funcName.hint != null) {
- switch (funcName.hint) {
+ if (funcName.hintToken != null) {
+ switch (funcName.hintToken.hint) {
case INDEXED_NESTED_LOOP_JOIN_HINT:
callExpr.addHint(IndexedNLJoinExpressionAnnotation.INSTANCE);
break;
case SKIP_SECONDARY_INDEX_SEARCH_HINT:
callExpr.addHint(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE);
break;
+ case RANGE_HINT:
+ try {
+ RangeAnnotation rangeAnn = new RangeAnnotation();
+ rangeAnn.setObject((Object) RangeMapBuilder.parseHint(parseExpression(funcName.hintToken.hintParams)));
+ callExpr.addHint(rangeAnn);
+ } catch (CompilationException e) {
+ {
+ SqlppParseException e2 = new SqlppParseException(getSourceLocation(funcName.hintToken), e.getMessage());
+ e2.initCause(e);
+ throw e2;
+ }
+ }
+ break;
}
}
FunctionMapUtil.normalizedListInputFunctions(callExpr);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
index 2178b48..58495e8 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalLogic.java
@@ -25,7 +25,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-class IntervalLogic {
+public class IntervalLogic {
private final IBinaryComparator comp;
private final ArrayBackedValueStorage s1 = new ArrayBackedValueStorage();
@@ -33,7 +33,7 @@
private final ArrayBackedValueStorage s2 = new ArrayBackedValueStorage();
private final ArrayBackedValueStorage e2 = new ArrayBackedValueStorage();
- IntervalLogic() {
+ public IntervalLogic() {
comp = BinaryComparatorFactoryProvider.INSTANCE
.getBinaryComparatorFactory(BuiltinType.ANY, BuiltinType.ANY, true).createBinaryComparator();
}
@@ -47,7 +47,7 @@
* @throws HyracksDataException IOException
* @see #after(AIntervalPointable, AIntervalPointable)
*/
- boolean before(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean before(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
e1.reset();
s2.reset();
ip1.getTaggedEnd(e1.getDataOutput());
@@ -56,7 +56,7 @@
s2.getStartOffset(), s2.getLength()) < 0;
}
- boolean after(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean after(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return before(ip2, ip1);
}
@@ -69,7 +69,7 @@
* @throws HyracksDataException IOException
* @see #metBy(AIntervalPointable, AIntervalPointable)
*/
- boolean meets(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean meets(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
e1.reset();
s2.reset();
ip1.getTaggedEnd(e1.getDataOutput());
@@ -78,7 +78,7 @@
s2.getStartOffset(), s2.getLength()) == 0;
}
- boolean metBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean metBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return meets(ip2, ip1);
}
@@ -91,7 +91,7 @@
* @throws HyracksDataException IOException
* @see #overlappedBy(AIntervalPointable, AIntervalPointable)
*/
- boolean overlaps(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean overlaps(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
s1.reset();
e1.reset();
s2.reset();
@@ -108,7 +108,7 @@
e2.getStartOffset(), e2.getLength()) < 0;
}
- boolean overlappedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean overlappedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return overlaps(ip2, ip1);
}
@@ -120,7 +120,7 @@
* @throws HyracksDataException IOException
* @return boolean
*/
- boolean overlapping(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean overlapping(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
s1.reset();
e1.reset();
s2.reset();
@@ -144,7 +144,7 @@
* @throws HyracksDataException IOException
* @see #startedBy(AIntervalPointable, AIntervalPointable)
*/
- boolean starts(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean starts(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
s1.reset();
e1.reset();
s2.reset();
@@ -159,7 +159,7 @@
e2.getStartOffset(), e2.getLength()) <= 0;
}
- boolean startedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean startedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return starts(ip2, ip1);
}
@@ -172,7 +172,7 @@
* @throws HyracksDataException IOException
* @see #coveredBy(AIntervalPointable, AIntervalPointable)
*/
- boolean covers(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean covers(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
s1.reset();
e1.reset();
s2.reset();
@@ -187,7 +187,7 @@
e2.getStartOffset(), e2.getLength()) >= 0;
}
- boolean coveredBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean coveredBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return covers(ip2, ip1);
}
@@ -200,7 +200,7 @@
* @throws HyracksDataException IOException
* @see #endedBy(AIntervalPointable, AIntervalPointable)
*/
- boolean ends(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean ends(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
s1.reset();
e1.reset();
s2.reset();
@@ -215,7 +215,7 @@
e2.getStartOffset(), e2.getLength()) == 0;
}
- boolean endedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
+ public boolean endedBy(AIntervalPointable ip1, AIntervalPointable ip2) throws HyracksDataException {
return ends(ip2, ip1);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java
new file mode 100644
index 0000000..f28faf1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoinOperatorDescriptor.java
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators.joins.interval;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtil;
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtilFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivity;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class IntervalMergeJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private static final int JOIN_BUILD_ACTIVITY_ID = 0;
+ private static final int JOIN_PROBE_ACTIVITY_ID = 1;
+ private final int[] leftKeys;
+ private final int[] rightKeys;
+ private final int memoryForJoin;
+ private final IIntervalJoinUtilFactory imjcf;
+
+ private final int probeKey;
+ private final int buildKey;
+
+ public IntervalMergeJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin, int[] leftKeys,
+ int[] rightKeys, RecordDescriptor recordDescriptor, IIntervalJoinUtilFactory imjcf) {
+ super(spec, 2, 1);
+ outRecDescs[0] = recordDescriptor;
+ this.buildKey = leftKeys[0];
+ this.probeKey = rightKeys[0];
+ this.leftKeys = leftKeys;
+ this.rightKeys = rightKeys;
+ this.memoryForJoin = memoryForJoin;
+ this.imjcf = imjcf;
+ }
+
+ @Override
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ ActivityId buildAid = new ActivityId(odId, JOIN_BUILD_ACTIVITY_ID);
+ ActivityId probeAid = new ActivityId(odId, JOIN_PROBE_ACTIVITY_ID);
+
+ IActivity probeAN = new JoinProbeActivityNode(probeAid);
+ IActivity buildAN = new JoinBuildActivityNode(buildAid, probeAid);
+
+ builder.addActivity(this, buildAN);
+ builder.addSourceEdge(0, buildAN, 0);
+
+ builder.addActivity(this, probeAN);
+ builder.addSourceEdge(1, probeAN, 0);
+ builder.addTargetEdge(0, probeAN, 0);
+ builder.addBlockingEdge(buildAN, probeAN);
+ }
+
+ public static class JoinCacheTaskState extends AbstractStateObject {
+ private IntervalMergeJoiner joiner;
+
+ private JoinCacheTaskState(JobId jobId, TaskId taskId) {
+ super(jobId, taskId);
+ }
+ }
+
+ private class JoinBuildActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityId nljAid;
+
+ public JoinBuildActivityNode(ActivityId id, ActivityId nljAid) {
+ super(id);
+ this.nljAid = nljAid;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+ private JoinCacheTaskState state;
+
+ @Override
+ public void open() throws HyracksDataException {
+ state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(),
+ new TaskId(getActivityId(), partition));
+
+ IIntervalJoinUtil imjc =
+ imjcf.createIntervalMergeJoinChecker(leftKeys, rightKeys, ctx, nPartitions);
+
+ state.joiner = new IntervalMergeJoiner(ctx, memoryForJoin, imjc, buildKey, probeKey, rd0, rd1);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+ FrameUtils.copyAndFlip(buffer, copyBuffer);
+ state.joiner.processBuildFrame(copyBuffer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ state.joiner.processBuildClose();
+ ctx.setStateObject(state);
+ }
+
+ @Override
+ public void fail() {
+ // No variables to update.
+ }
+ };
+ }
+ }
+
+ private class JoinProbeActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ public JoinProbeActivityNode(ActivityId id) {
+ super(id);
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+ return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+ private JoinCacheTaskState state;
+
+ @Override
+ public void open() throws HyracksDataException {
+ writer.open();
+ state = (JoinCacheTaskState) ctx.getStateObject(
+ new TaskId(new ActivityId(getOperatorId(), JOIN_BUILD_ACTIVITY_ID), partition));
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ state.joiner.processProbeFrame(buffer, writer);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ state.joiner.processProbeClose(writer);
+ } finally {
+ writer.close();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ writer.fail();
+ }
+ };
+ }
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoiner.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoiner.java
new file mode 100644
index 0000000..13d8c25
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/IntervalMergeJoiner.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.LinkedList;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtil;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.ITupleAccessor;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalSideTuple;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalVariableDeletableTupleMemoryManager;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFilePointer;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFileStream;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.TupleAccessor;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.buffermanager.DeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+/**
+ * Merge Joiner takes two sorted streams of input and joins.
+ * The two sorted streams must be in a logical order and the comparator must
+ * support keeping that order so the join will work.
+ * The left stream will spill to disk when memory is full.
+ * The right stream spills to memory and pause when memory is full.
+ */
+public class IntervalMergeJoiner {
+
+ public enum TupleStatus {
+ LOADED,
+ EMPTY;
+
+ public boolean isLoaded() {
+ return this.equals(LOADED);
+ }
+
+ public boolean isEmpty() {
+ return this.equals(EMPTY);
+ }
+ }
+
+ private final IDeallocatableFramePool framePool;
+ private final IDeletableTupleBufferManager bufferManager;
+ private final ITupleAccessor memoryAccessor;
+ private final LinkedList<TuplePointer> memoryBuffer = new LinkedList<>();
+
+ private final RunFileStream runFileStream;
+ private final RunFilePointer runFilePointer;
+
+ private IntervalSideTuple memoryTuple;
+ private IntervalSideTuple[] inputTuple;
+
+ private final IIntervalJoinUtil mjc;
+
+ protected static final int JOIN_PARTITIONS = 2;
+ protected static final int BUILD_PARTITION = 0;
+ protected static final int PROBE_PARTITION = 1;
+
+ protected final IFrame[] inputBuffer;
+ protected final FrameTupleAppender resultAppender;
+ protected final ITupleAccessor[] inputAccessor;
+
+ protected long[] frameCounts = { 0, 0 };
+ protected long[] tupleCounts = { 0, 0 };
+
+ public IntervalMergeJoiner(IHyracksTaskContext ctx, int memorySize, IIntervalJoinUtil mjc, int buildKeys,
+ int probeKeys, RecordDescriptor buildRd, RecordDescriptor probeRd) throws HyracksDataException {
+ this.mjc = mjc;
+
+ // Memory (probe buffer)
+ if (memorySize < 1) {
+ throw new HyracksDataException(
+ "MergeJoiner does not have enough memory (needs > 0, got " + memorySize + ").");
+ }
+
+ inputAccessor = new TupleAccessor[JOIN_PARTITIONS];
+ inputAccessor[BUILD_PARTITION] = new TupleAccessor(buildRd);
+ inputAccessor[PROBE_PARTITION] = new TupleAccessor(probeRd);
+
+ inputBuffer = new IFrame[JOIN_PARTITIONS];
+ inputBuffer[BUILD_PARTITION] = new VSizeFrame(ctx);
+ inputBuffer[PROBE_PARTITION] = new VSizeFrame(ctx);
+
+ framePool = new DeallocatableFramePool(ctx, (memorySize) * ctx.getInitialFrameSize());
+ bufferManager = new IntervalVariableDeletableTupleMemoryManager(framePool, probeRd);
+ memoryAccessor = ((IntervalVariableDeletableTupleMemoryManager) bufferManager).createTupleAccessor();
+
+ // Run File and frame cache (build buffer)
+ runFileStream = new RunFileStream(ctx, "ismj-left");
+ runFilePointer = new RunFilePointer();
+ runFileStream.createRunFileWriting();
+ runFileStream.startRunFileWriting();
+
+ memoryTuple = new IntervalSideTuple(mjc, memoryAccessor, probeKeys);
+
+ inputTuple = new IntervalSideTuple[JOIN_PARTITIONS];
+ inputTuple[PROBE_PARTITION] = new IntervalSideTuple(mjc, inputAccessor[PROBE_PARTITION], probeKeys);
+ inputTuple[BUILD_PARTITION] = new IntervalSideTuple(mjc, inputAccessor[BUILD_PARTITION], buildKeys);
+
+ // Result
+ this.resultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+ }
+
+ public void processBuildFrame(ByteBuffer buffer) throws HyracksDataException {
+ inputAccessor[BUILD_PARTITION].reset(buffer);
+ for (int x = 0; x < inputAccessor[BUILD_PARTITION].getTupleCount(); x++) {
+ runFileStream.addToRunFile(inputAccessor[BUILD_PARTITION], x);
+ }
+ }
+
+ public void processBuildClose() throws HyracksDataException {
+ runFileStream.flushRunFile();
+ runFileStream.startReadingRunFile(inputAccessor[BUILD_PARTITION]);
+ }
+
+ public void processProbeFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+ inputAccessor[PROBE_PARTITION].reset(buffer);
+ inputAccessor[PROBE_PARTITION].next();
+
+ TupleStatus buildTs = loadBuildTuple();
+ TupleStatus probeTs = loadProbeTuple();
+ while (buildTs.isLoaded() && (probeTs.isLoaded() || memoryHasTuples())) {
+ if (probeTs.isLoaded()) {
+ // Right side from stream
+ processProbeTuple(writer);
+ probeTs = loadProbeTuple();
+ } else {
+ // Left side from stream
+ processBuildTuple(writer);
+ buildTs = loadBuildTuple();
+ }
+ }
+ }
+
+ public void processProbeClose(IFrameWriter writer) throws HyracksDataException {
+ resultAppender.write(writer, true);
+ runFileStream.close();
+ runFileStream.removeRunFile();
+ }
+
+ private TupleStatus loadProbeTuple() {
+ TupleStatus loaded;
+ if (inputAccessor[PROBE_PARTITION] != null && inputAccessor[PROBE_PARTITION].exists()) {
+ // Still processing frame.
+ loaded = TupleStatus.LOADED;
+ } else {
+ // No more frames or tuples to process.
+ loaded = TupleStatus.EMPTY;
+ }
+ return loaded;
+ }
+
+ private TupleStatus loadBuildTuple() throws HyracksDataException {
+ if (!inputAccessor[BUILD_PARTITION].exists()) {
+ // Must keep condition in a separate if due to actions applied in loadNextBuffer.
+ if (!runFileStream.loadNextBuffer(inputAccessor[BUILD_PARTITION])) {
+ return TupleStatus.EMPTY;
+ }
+ }
+ return TupleStatus.LOADED;
+ }
+
+ private void processBuildTuple(IFrameWriter writer) throws HyracksDataException {
+ // Check against memory
+ if (memoryHasTuples()) {
+ inputTuple[BUILD_PARTITION].loadTuple();
+ Iterator<TuplePointer> memoryIterator = memoryBuffer.iterator();
+ while (memoryIterator.hasNext()) {
+ TuplePointer tp = memoryIterator.next();
+ memoryTuple.setTuple(tp);
+ if (inputTuple[BUILD_PARTITION].removeFromMemory(memoryTuple)) {
+ // remove from memory
+ bufferManager.deleteTuple(tp);
+ memoryIterator.remove();
+ continue;
+ } else if (inputTuple[BUILD_PARTITION].checkForEarlyExit(memoryTuple)) {
+ // No more possible comparisons
+ break;
+ } else if (inputTuple[BUILD_PARTITION].compareJoin(memoryTuple)) {
+ // add to result
+ addToResult(inputAccessor[BUILD_PARTITION], inputAccessor[BUILD_PARTITION].getTupleId(),
+ memoryAccessor, tp.getTupleIndex(), writer);
+ }
+ }
+ }
+ inputAccessor[BUILD_PARTITION].next();
+ }
+
+ private void processProbeTuple(IFrameWriter writer) throws HyracksDataException {
+ // append to memory
+ if (mjc.checkToSaveInMemory(inputAccessor[BUILD_PARTITION], inputAccessor[BUILD_PARTITION].getTupleId(),
+ inputAccessor[PROBE_PARTITION], inputAccessor[PROBE_PARTITION].getTupleId())) {
+ if (!addToMemory(inputAccessor[PROBE_PARTITION])) {
+ unfreezeAndClearMemory(writer, inputAccessor[BUILD_PARTITION]);
+ return;
+ }
+ }
+ inputAccessor[PROBE_PARTITION].next();
+ }
+
+ private void unfreezeAndClearMemory(IFrameWriter writer, ITupleAccessor accessor) throws HyracksDataException {
+ runFilePointer.reset(runFileStream.getReadPointer(), inputAccessor[BUILD_PARTITION].getTupleId());
+ TupleStatus buildTs = loadBuildTuple();
+ while (buildTs.isLoaded() && memoryHasTuples()) {
+ // Left side from stream
+ processBuildTuple(writer);
+ buildTs = loadBuildTuple();
+ }
+ // Finish writing
+ runFileStream.flushRunFile();
+ // Clear memory
+ memoryBuffer.clear();
+ bufferManager.reset();
+ // Start reading
+ runFileStream.startReadingRunFile(accessor, runFilePointer.getFileOffset());
+ accessor.setTupleId(runFilePointer.getTupleIndex());
+ runFilePointer.reset(-1, -1);
+ }
+
+ private boolean addToMemory(ITupleAccessor accessor) throws HyracksDataException {
+ TuplePointer tp = new TuplePointer();
+ if (bufferManager.insertTuple(accessor, accessor.getTupleId(), tp)) {
+ memoryBuffer.add(tp);
+ return true;
+ }
+ return false;
+ }
+
+ private void addToResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex, IFrameWriter writer) throws HyracksDataException {
+ FrameUtils.appendConcatToWriter(writer, resultAppender, accessorLeft, leftTupleIndex, accessorRight,
+ rightTupleIndex);
+ }
+
+ private boolean memoryHasTuples() {
+ return bufferManager.getNumTuples() > 0;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalInverseJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalInverseJoinUtil.java
new file mode 100644
index 0000000..7830b0c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalInverseJoinUtil.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+
+public abstract class AbstractIntervalInverseJoinUtil extends AbstractIntervalJoinUtil {
+
+ public AbstractIntervalInverseJoinUtil(int idLeft, int idRight) {
+ super(idLeft, idRight);
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 < end1;
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 >= end1;
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalJoinUtil.java
new file mode 100644
index 0000000..da50ab2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AbstractIntervalJoinUtil.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.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalLogic;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+
+public abstract class AbstractIntervalJoinUtil implements IIntervalJoinUtil {
+
+ protected final int idLeft;
+ protected final int idRight;
+
+ protected final IntervalLogic il = new IntervalLogic();
+
+ protected final TaggedValuePointable tvp = TaggedValuePointable.FACTORY.createPointable();
+ protected final AIntervalPointable ipLeft = (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
+ protected final AIntervalPointable ipRight = (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
+
+ protected final IBinaryComparator ch = BinaryComparatorFactoryProvider.INSTANCE
+ .getBinaryComparatorFactory(BuiltinType.ANY, BuiltinType.ANY, true).createBinaryComparator();
+ protected final IPointable startLeft = VoidPointable.FACTORY.createPointable();
+ protected final IPointable startRight = VoidPointable.FACTORY.createPointable();
+
+ public AbstractIntervalJoinUtil(int idLeft, int idRight) {
+ this.idLeft = idLeft;
+ this.idRight = idRight;
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ return start0 <= start1;
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+ return start0 > start1;
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkIfMoreMatches(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+ long end0 = IntervalJoinUtil.getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+ return end0 > start1;
+ }
+
+ @Override
+ public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex, boolean reversed) throws HyracksDataException {
+ if (reversed) {
+ IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, ipRight);
+ IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, ipLeft);
+ } else {
+ IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, ipLeft);
+ IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, ipRight);
+ }
+ return compareInterval(ipLeft, ipRight);
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkForEarlyExit(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+ long end0 = IntervalJoinUtil.getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+ return end0 < start1;
+ }
+
+ @Override
+ public abstract boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight)
+ throws HyracksDataException;
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtil.java
new file mode 100644
index 0000000..e192943
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtil.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.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class AfterIntervalJoinUtil extends AbstractIntervalJoinUtil {
+
+ public AfterIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleID,
+ IFrameTupleAccessor accessorRight, int rightTupleID) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleID, idRight);
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleID, idRight);
+ return start0 >= start1;
+ }
+
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return false;
+ }
+
+ @Override
+ public boolean checkForEarlyExit(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return false;
+ }
+
+ @Override
+ public boolean checkIfMoreMatches(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return true;
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.after(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..b54edf8
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/AfterIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class AfterIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new AfterIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtil.java
new file mode 100644
index 0000000..304775c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtil.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.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class BeforeIntervalJoinUtil extends AbstractIntervalJoinUtil {
+
+ public BeforeIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleID,
+ IFrameTupleAccessor accessorRight, int rightTupleID) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleID, idRight);
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleID, idRight);
+ return start0 <= start1;
+ }
+
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return false;
+ }
+
+ @Override
+ public boolean checkForEarlyExit(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return false;
+ }
+
+ @Override
+ public boolean checkIfMoreMatches(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ return true;
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.before(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..a653ff9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/BeforeIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class BeforeIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new BeforeIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtil.java
new file mode 100644
index 0000000..a8e8177
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtil.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CoveredByIntervalJoinUtil extends AbstractIntervalInverseJoinUtil {
+
+ public CoveredByIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 <= end1;
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 > end1;
+
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.coveredBy(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..7f52c11
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoveredByIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class CoveredByIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new CoveredByIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtil.java
new file mode 100644
index 0000000..cdfaafd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtil.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CoversIntervalJoinUtil extends AbstractIntervalJoinUtil {
+
+ public CoversIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.covers(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..9b7481e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/CoversIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class CoversIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new CoversIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtil.java
new file mode 100644
index 0000000..91c14b0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtil.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IIntervalJoinUtil {
+
+ /**
+ * Check to see if the right tuple should be added to memory during the merge join.
+ * The memory is used to check the right tuple with the remaining left tuples.
+ * The check is true if the next left tuple could still match with this right tuple.
+ *
+ * @param accessorLeft
+ * @param accessorRight
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if the right tuple should be removed from memory during the merge join.
+ * The memory is used to check the right tuple with the remaining left tuples.
+ * The check is true if the next left tuple is NOT able match with this right tuple.
+ *
+ * @param accessorLeft
+ * @param accessorRight
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if the left tuple should continue checking for matches.
+ * The check is true if the next left tuple is NOT able match with this right tuple.
+ *
+ * @param accessorLeft
+ * @param accessorRight
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkIfMoreMatches(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex) throws HyracksDataException;
+
+ /**
+ * Check to see if tuples match join condition
+ * The check is true if they match.
+ *
+ * @param accessorLeft
+ * @param leftTupleIndex
+ * @param accessorRight
+ * @param rightTupleIndex
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex, boolean reversed) throws HyracksDataException;
+
+ /**
+ * Check to see if the interval matches the join condition.
+ * The check is true if it matches.
+ *
+ * @param ipLeft
+ * @param ipRight
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException;
+
+ /**
+ * Check to see if the left tuple should stop checking for matches.
+ * The check is true if there can be no more matches
+ *
+ * @param accessorLeft
+ * @param accessorRight
+ * @return boolean
+ * @throws HyracksDataException
+ */
+ boolean checkForEarlyExit(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex) throws HyracksDataException;
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..b5f0bba
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/IIntervalJoinUtilFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IIntervalJoinUtilFactory extends Serializable {
+
+ IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx, int nPartitions)
+ throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtil.java
new file mode 100644
index 0000000..e2e79f9
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtil.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlappedByIntervalJoinUtil extends AbstractIntervalInverseJoinUtil {
+
+ public OverlappedByIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.overlappedBy(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..74de359
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappedByIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class OverlappedByIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new OverlappedByIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtil.java
new file mode 100644
index 0000000..10a182c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtil.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlappingIntervalJoinUtil extends AbstractIntervalJoinUtil {
+ private final long partitionStart;
+
+ public OverlappingIntervalJoinUtil(int[] keysLeft, int[] keysRight, long partitionStart) {
+ super(keysLeft[0], keysRight[0]);
+ this.partitionStart = partitionStart;
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 < end1;
+ }
+
+ /**
+ * Right (second argument) interval starts before left (first argument) interval ends.
+ */
+ @Override
+ public boolean checkForEarlyExit(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+ long end0 = IntervalJoinUtil.getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+ return end0 <= start1;
+ }
+
+ /**
+ * Left (first argument) interval starts after the Right (second argument) interval ends.
+ */
+ @Override
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) {
+ long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+ long end1 = IntervalJoinUtil.getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+ return start0 >= end1;
+ }
+
+ @Override
+ public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex, boolean reversed) throws HyracksDataException {
+ if (reversed) {
+ IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, ipRight);
+ IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, ipLeft);
+ } else {
+ IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, ipLeft);
+ IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, ipRight);
+ }
+ if (ipLeft.getStartValue() < partitionStart && ipRight.getStartValue() < partitionStart) {
+ // These tuples match in a different partition
+ return false;
+ }
+ return compareInterval(ipLeft, ipRight);
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.overlapping(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..a1f3203
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlappingIntervalJoinUtilFactory.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.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class OverlappingIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+ private final RangeMap rangeMap;
+
+ public OverlappingIntervalJoinUtilFactory(RangeMap rangeMap) {
+ this.rangeMap = rangeMap;
+ }
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) throws HyracksDataException {
+ int fieldIndex = 0;
+ int partition = ctx.getTaskAttemptId().getTaskId().getPartition();
+ //Calculate Partitions slot
+ int nRanges = rangeMap.getSplitCount() + 1;
+ double rangesPerPart = 1.0;
+ if (nRanges > nPartitions) {
+ rangesPerPart = ((double) nRanges) / nPartitions;
+ }
+ int slot = ((int) Math.ceil(partition * rangesPerPart) % nRanges) - 1;
+ //Find Partitions Start Value based on slot
+ long partitionStart = Long.MIN_VALUE;
+ if (slot >= 0) {
+ switch (ATypeTag.VALUE_TYPE_MAPPING[rangeMap.getTag(fieldIndex, slot)]) {
+ case DATETIME:
+ partitionStart = ADateTimeSerializerDeserializer.getChronon(rangeMap.getByteArray(),
+ rangeMap.getStartOffset(fieldIndex, slot) + 1);
+ break;
+ case DATE:
+ partitionStart = ADateSerializerDeserializer.getChronon(rangeMap.getByteArray(),
+ rangeMap.getStartOffset(fieldIndex, slot) + 1);
+ break;
+ case TIME:
+ partitionStart = ATimeSerializerDeserializer.getChronon(rangeMap.getByteArray(),
+ rangeMap.getStartOffset(fieldIndex, slot) + 1);
+ break;
+ default:
+ throw new HyracksDataException("RangeMap type is not supported");
+ }
+ }
+ return new OverlappingIntervalJoinUtil(keys0, keys1, partitionStart);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtil.java
new file mode 100644
index 0000000..392c0cf
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtil.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlapsIntervalJoinUtil extends AbstractIntervalJoinUtil {
+
+ public OverlapsIntervalJoinUtil(int[] keysLeft, int[] keysRight) {
+ super(keysLeft[0], keysRight[0]);
+ }
+
+ @Override
+ public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+ return il.overlaps(ipLeft, ipRight);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtilFactory.java
new file mode 100644
index 0000000..fb8be0b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/OverlapsIntervalJoinUtilFactory.java
@@ -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.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class OverlapsIntervalJoinUtilFactory implements IIntervalJoinUtilFactory {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IIntervalJoinUtil createIntervalMergeJoinChecker(int[] keys0, int[] keys1, IHyracksTaskContext ctx,
+ int nPartitions) {
+ return new OverlapsIntervalJoinUtil(keys0, keys1);
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/AbstractTupleAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/AbstractTupleAccessor.java
new file mode 100644
index 0000000..397f959
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/AbstractTupleAccessor.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public abstract class AbstractTupleAccessor implements ITupleAccessor {
+ public static final int UNSET = -2;
+ public static final int INITIALIZED = -1;
+
+ protected int tupleId = UNSET;
+
+ protected int frameId;
+
+ protected abstract IFrameTupleAccessor getInnerAccessor();
+
+ protected abstract void resetInnerAccessor(int frameId);
+
+ protected abstract void resetInnerAccessor(TuplePointer tp);
+
+ protected abstract int getFrameCount();
+
+ @Override
+ public int getTupleStartOffset() {
+ return getTupleStartOffset(tupleId);
+ }
+
+ @Override
+ public int getTupleLength() {
+ return getTupleLength(tupleId);
+ }
+
+ @Override
+ public int getAbsFieldStartOffset(int fieldId) {
+ return getAbsoluteFieldStartOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public int getFieldLength(int fieldId) {
+ return getFieldLength(tupleId, fieldId);
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return getInnerAccessor().getBuffer();
+ }
+
+ @Override
+ public int getFieldCount() {
+ return getInnerAccessor().getFieldCount();
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return getInnerAccessor().getFieldSlotsLength();
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return getInnerAccessor().getFieldEndOffset(tupleId, fIdx);
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return getInnerAccessor().getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getInnerAccessor().getFieldLength(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ return getInnerAccessor().getTupleLength(tupleIndex);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return getInnerAccessor().getTupleEndOffset(tupleIndex);
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return getInnerAccessor().getTupleStartOffset(tupleIndex);
+ }
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+ return getInnerAccessor().getAbsoluteFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return getInnerAccessor().getTupleCount();
+ }
+
+ @Override
+ public void reset(TuplePointer tuplePointer) {
+ resetInnerAccessor(tuplePointer.getFrameIndex());
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ throw new IllegalAccessError("Should never call this reset");
+ }
+
+ @Override
+ public int getTupleEndOffset() {
+ return getInnerAccessor().getTupleEndOffset(tupleId);
+ }
+
+ @Override
+ public int getFieldEndOffset(int fieldId) {
+ return getInnerAccessor().getFieldEndOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public int getFieldStartOffset(int fieldId) {
+ return getInnerAccessor().getFieldStartOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public void getTuplePointer(TuplePointer tp) {
+ tp.reset(frameId, tupleId);
+ }
+
+ @Override
+ public int getTupleId() {
+ return tupleId;
+ }
+
+ @Override
+ public void setTupleId(int tupleId) {
+ this.tupleId = tupleId;
+ }
+
+ @Override
+ public void reset() {
+ tupleId = INITIALIZED;
+ frameId = 0;
+ resetInnerAccessor(frameId);
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (tupleId + 1 < getTupleCount() || frameId + 1 < getFrameCount()) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public boolean exists() {
+ return INITIALIZED < tupleId && getTupleEndOffset(tupleId) > 0 && tupleId < getTupleCount()
+ && frameId < getFrameCount();
+ }
+
+ @Override
+ public void next() {
+ // TODO Consider error messages
+ if (tupleId + 1 < getTupleCount()) {
+ ++tupleId;
+ } else if (frameId + 1 < getFrameCount()) {
+ ++frameId;
+ resetInnerAccessor(frameId);
+ tupleId = 0;
+ } else {
+ // Force exists to fail, by incrementing the tuple pointer.
+ ++tupleId;
+ }
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/ITupleAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/ITupleAccessor.java
new file mode 100644
index 0000000..77059d6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/ITupleAccessor.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.runtime.operators.joins.interval.utils.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface ITupleAccessor extends IFrameTupleAccessor {
+ int getTupleStartOffset();
+
+ int getTupleEndOffset();
+
+ int getTupleLength();
+
+ int getAbsFieldStartOffset(int fieldId);
+
+ int getFieldLength(int fieldId);
+
+ @Override
+ int getFieldCount();
+
+ @Override
+ int getFieldSlotsLength();
+
+ int getFieldEndOffset(int fieldId);
+
+ int getFieldStartOffset(int fieldId);
+
+ void reset(TuplePointer tuplePointer);
+
+ @Override
+ void reset(ByteBuffer buffer);
+
+ int getTupleId();
+
+ void setTupleId(int tupleId);
+
+ void getTuplePointer(TuplePointer tp);
+
+ /**
+ * Only reset the iterator.
+ */
+ void reset();
+
+ boolean hasNext();
+
+ void next();
+
+ boolean exists();
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalJoinUtil.java
new file mode 100644
index 0000000..7ff5816
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalJoinUtil.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+
+public class IntervalJoinUtil {
+
+ private IntervalJoinUtil() {
+ }
+
+ public static void getIntervalPointable(ITupleAccessor accessor, int fieldId, TaggedValuePointable tvp,
+ AIntervalPointable ip) {
+ int start =
+ accessor.getTupleStartOffset() + accessor.getFieldSlotsLength() + accessor.getFieldStartOffset(fieldId);
+ int length = accessor.getFieldLength(fieldId);
+ tvp.set(accessor.getBuffer().array(), start, length);
+ tvp.getValue(ip);
+ }
+
+ public static void getIntervalPointable(IFrameTupleAccessor accessor, int tupleId, int fieldId,
+ AIntervalPointable ip) {
+ int start = getIntervalOffset(accessor, tupleId, fieldId);
+ int length = accessor.getFieldLength(tupleId, fieldId) - 1;
+ ip.set(accessor.getBuffer().array(), start, length);
+ }
+
+ public static int getIntervalOffset(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ return getIntervalOffsetWithTag(accessor, tupleId, fieldId) + 1;
+ }
+
+ public static int getIntervalOffsetWithTag(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+ + accessor.getFieldStartOffset(tupleId, fieldId);
+ return start;
+ }
+
+ public static long getIntervalStart(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ int start = getIntervalOffset(accessor, tupleId, fieldId);
+ long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), start);
+ return intervalStart;
+ }
+
+ public static long getIntervalEnd(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+ int start = getIntervalOffset(accessor, tupleId, fieldId);
+ long intervalEnd = AIntervalSerializerDeserializer.getIntervalEnd(accessor.getBuffer().array(), start);
+ return intervalEnd;
+ }
+
+ public static long getIntervalStart(ITupleAccessor accessor, int fieldId) {
+ int start = accessor.getTupleStartOffset() + accessor.getFieldSlotsLength()
+ + accessor.getFieldStartOffset(fieldId) + 1;
+ long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), start);
+ return intervalStart;
+ }
+
+ public static long getIntervalEnd(ITupleAccessor accessor, int fieldId) {
+ int start = accessor.getTupleStartOffset() + accessor.getFieldSlotsLength()
+ + accessor.getFieldStartOffset(fieldId) + 1;
+ long intervalEnd = AIntervalSerializerDeserializer.getIntervalEnd(accessor.getBuffer().array(), start);
+ return intervalEnd;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalSideTuple.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalSideTuple.java
new file mode 100644
index 0000000..cffc491
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalSideTuple.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.asterix.runtime.operators.joins.interval.utils.IIntervalJoinUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class IntervalSideTuple {
+ // Tuple access
+ int fieldId;
+ ITupleAccessor accessor;
+ int tupleIndex;
+ int frameIndex = -1;
+
+ long start;
+ long end;
+
+ // Join details
+ final IIntervalJoinUtil imjc;
+
+ public IntervalSideTuple(IIntervalJoinUtil imjc, ITupleAccessor accessor, int fieldId) {
+ this.imjc = imjc;
+ this.accessor = accessor;
+ this.fieldId = fieldId;
+ }
+
+ public void setTuple(TuplePointer tp) {
+ if (frameIndex != tp.getFrameIndex()) {
+ accessor.reset(tp);
+ frameIndex = tp.getFrameIndex();
+ }
+ tupleIndex = tp.getTupleIndex();
+ int offset = IntervalJoinUtil.getIntervalOffset(accessor, tupleIndex, fieldId);
+ start = AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), offset);
+ end = AIntervalSerializerDeserializer.getIntervalEnd(accessor.getBuffer().array(), offset);
+ }
+
+ public void loadTuple() {
+ tupleIndex = accessor.getTupleId();
+ int offset = IntervalJoinUtil.getIntervalOffset(accessor, tupleIndex, fieldId);
+ start = AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), offset);
+ end = AIntervalSerializerDeserializer.getIntervalEnd(accessor.getBuffer().array(), offset);
+ }
+
+ public int getTupleIndex() {
+ return tupleIndex;
+ }
+
+ public ITupleAccessor getAccessor() {
+ return accessor;
+ }
+
+ public long getStart() {
+ return start;
+ }
+
+ public long getEnd() {
+ return end;
+ }
+
+ public boolean hasMoreMatches(IntervalSideTuple ist) throws HyracksDataException {
+ return imjc.checkIfMoreMatches(accessor, tupleIndex, ist.accessor, ist.tupleIndex);
+ }
+
+ public boolean compareJoin(IntervalSideTuple ist) throws HyracksDataException {
+ return imjc.checkToSaveInResult(accessor, tupleIndex, ist.accessor, ist.tupleIndex, false);
+ }
+
+ public boolean removeFromMemory(IntervalSideTuple ist) throws HyracksDataException {
+ return imjc.checkToRemoveInMemory(accessor, tupleIndex, ist.accessor, ist.tupleIndex);
+ }
+
+ public boolean checkForEarlyExit(IntervalSideTuple ist) throws HyracksDataException {
+ return imjc.checkForEarlyExit(accessor, tupleIndex, ist.accessor, ist.tupleIndex);
+ }
+
+ public boolean startsBefore(IntervalSideTuple ist) {
+ return start <= ist.start;
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalVariableDeletableTupleMemoryManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalVariableDeletableTupleMemoryManager.java
new file mode 100644
index 0000000..aebb932
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/IntervalVariableDeletableTupleMemoryManager.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.std.buffermanager.IFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableDeletableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.sort.util.DeletableFrameTupleAppender;
+import org.apache.hyracks.dataflow.std.sort.util.IAppendDeletableFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class IntervalVariableDeletableTupleMemoryManager extends VariableDeletableTupleMemoryManager {
+ public IntervalVariableDeletableTupleMemoryManager(IFramePool framePool, RecordDescriptor recordDescriptor) {
+ super(framePool, recordDescriptor);
+ }
+
+ public ITupleAccessor createTupleAccessor() {
+ return new AbstractTupleAccessor() {
+ private IAppendDeletableFrameTupleAccessor bufferAccessor =
+ new DeletableFrameTupleAppender(recordDescriptor);
+
+ @Override
+ protected IFrameTupleAccessor getInnerAccessor() {
+ return bufferAccessor;
+ }
+
+ protected void resetInnerAccessor(TuplePointer tuplePointer) {
+ bufferAccessor.reset(frames.get(tuplePointer.getFrameIndex()));
+ }
+
+ @Override
+ protected void resetInnerAccessor(int frameIndex) {
+ bufferAccessor.reset(frames.get(frameIndex));
+ }
+
+ @Override
+ protected int getFrameCount() {
+ return frames.size();
+ }
+
+ @Override
+ public boolean hasNext() {
+ return hasNext(frameId, tupleId);
+ }
+
+ @Override
+ public void next() {
+ tupleId = nextTuple(frameId, tupleId);
+ if (tupleId > INITIALIZED) {
+ return;
+ }
+
+ if (frameId + 1 < getFrameCount()) {
+ ++frameId;
+ resetInnerAccessor(frameId);
+ tupleId = INITIALIZED;
+ next();
+ }
+ }
+
+ public boolean hasNext(int fId, int tId) {
+ int id = nextTuple(fId, tId);
+ if (id > INITIALIZED) {
+ return true;
+ }
+ if (fId + 1 < getFrameCount()) {
+ return hasNext(fId + 1, INITIALIZED);
+ }
+ return false;
+ }
+
+ public int nextTuple(int fId, int tId) {
+ if (fId != frameId) {
+ resetInnerAccessor(fId);
+ }
+ int id = nextTupleInFrame(tId);
+ if (fId != frameId) {
+ resetInnerAccessor(frameId);
+ }
+ return id;
+ }
+
+ public int nextTupleInFrame(int tId) {
+ int id = tId;
+ while (id + 1 < getTupleCount()) {
+ ++id;
+ if (getTupleEndOffset(id) > 0) {
+ return id;
+ }
+ }
+ return UNSET;
+ }
+ };
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFilePointer.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFilePointer.java
new file mode 100644
index 0000000..1247948
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFilePointer.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import java.util.Comparator;
+
+import org.apache.hyracks.dataflow.std.structures.IResetable;
+
+public final class RunFilePointer implements IResetable<RunFilePointer> {
+ public static final int INVALID_ID = -1;
+ private long fileOffset;
+ private int tupleIndex;
+
+ public static final Comparator<RunFilePointer> ASC = (tp1, tp2) -> {
+ int c = (int) (tp1.getFileOffset() - tp2.getFileOffset());
+ if (c == 0) {
+ c = tp1.getTupleIndex() - tp2.getTupleIndex();
+ }
+ return c;
+ };
+
+ public static final Comparator<RunFilePointer> DESC = (tp1, tp2) -> {
+ int c = (int) (tp2.getFileOffset() - tp1.getFileOffset());
+ if (c == 0) {
+ c = tp2.getTupleIndex() - tp1.getTupleIndex();
+ }
+ return c;
+ };
+
+ public RunFilePointer() {
+ this(INVALID_ID, INVALID_ID);
+ }
+
+ public RunFilePointer(long fileOffset, int tupleId) {
+ reset(fileOffset, tupleId);
+ }
+
+ public long getFileOffset() {
+ return fileOffset;
+ }
+
+ public int getTupleIndex() {
+ return tupleIndex;
+ }
+
+ @Override
+ public void reset(RunFilePointer other) {
+ reset(other.fileOffset, other.tupleIndex);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ } else if (o == null || getClass() != o.getClass()) {
+ return false;
+ } else {
+ final RunFilePointer that = (RunFilePointer) o;
+ return fileOffset == that.fileOffset && tupleIndex == that.tupleIndex;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ int result = (int) fileOffset;
+ result = 31 * result + tupleIndex;
+ return result;
+ }
+
+ public void reset(long fileOffset, int tupleId) {
+ this.fileOffset = fileOffset;
+ this.tupleIndex = tupleId;
+ }
+
+ @Override
+ public String toString() {
+ return "RunFilePointer(" + fileOffset + ", " + tupleIndex + ")";
+ }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFileStream.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFileStream.java
new file mode 100644
index 0000000..fafe12e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/RunFileStream.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameTupleAppender;
+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.io.FileReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+
+public class RunFileStream {
+
+ private final String key;
+ private final IFrame runFileBuffer;
+ private final IFrameTupleAppender runFileAppender;
+ private RunFileWriter runFileWriter;
+ private RunFileReader runFileReader;
+ private FileReference runfile;
+
+ private final IHyracksTaskContext ctx;
+
+ private long runFileCounter = 0;
+ private long readCount = 0;
+ private long writeCount = 0;
+ private long totalTupleCount = 0;
+ private long previousReadPointer;
+
+ private boolean reading = false;
+ private boolean writing = false;
+
+ /**
+ * The RunFileSream uses two frames to buffer read and write operations.
+ * WorkFlow: CreateRunFileWriter, Write information, close it, flush it,
+ * go to the next frame, and repeat.
+ *
+ * @param ctx
+ * @param key
+ * @throws HyracksDataException
+ */
+ public RunFileStream(IHyracksTaskContext ctx, String key) throws HyracksDataException {
+ this.ctx = ctx;
+ this.key = key;
+
+ // TODO make the stream only use one buffer.
+ runFileBuffer = new VSizeFrame(ctx);
+ runFileAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+
+ }
+
+ public long getFileCount() {
+ return runFileCounter;
+ }
+
+ public long getTupleCount() {
+ return totalTupleCount;
+ }
+
+ public long getReadCount() {
+ return readCount;
+ }
+
+ public long getWriteCount() {
+ return writeCount;
+ }
+
+ public void createRunFileWriting() throws HyracksDataException {
+ runFileCounter++;
+ String prefix = key + '-' + runFileCounter + '-' + this.toString();
+ runfile = ctx.getJobletContext().createManagedWorkspaceFile(prefix);
+ if (runFileWriter != null) {
+ runFileWriter.close();
+ }
+
+ runFileWriter = new RunFileWriter(runfile, ctx.getIoManager());
+ runFileWriter.open();
+ totalTupleCount = 0;
+ }
+
+ public void startRunFileWriting() throws HyracksDataException {
+ writing = true;
+ runFileBuffer.reset();
+ }
+
+ public void addToRunFile(ITupleAccessor accessor) throws HyracksDataException {
+ int idx = accessor.getTupleId();
+ addToRunFile(accessor, idx);
+ }
+
+ public void addToRunFile(IFrameTupleAccessor accessor, int idx) throws HyracksDataException {
+ if (!runFileAppender.append(accessor, idx)) {
+ runFileAppender.write(runFileWriter, true);
+ writeCount++;
+ runFileAppender.append(accessor, idx);
+ }
+ totalTupleCount++;
+ }
+
+ public void startReadingRunFile(ITupleAccessor accessor) throws HyracksDataException {
+ startReadingRunFile(accessor, 0);
+ }
+
+ public void startReadingRunFile(ITupleAccessor accessor, long startOffset) throws HyracksDataException {
+ if (runFileReader != null) {
+ runFileReader.close();
+ }
+ reading = true;
+ // Create reader
+ runFileReader = runFileWriter.createReader();
+ runFileReader.open();
+ runFileReader.seek(startOffset);
+ previousReadPointer = 0;
+ // Load first frame
+ loadNextBuffer(accessor);
+ }
+
+ public boolean loadNextBuffer(ITupleAccessor accessor) throws HyracksDataException {
+ final long tempFrame = runFileReader.position();
+ if (runFileReader.nextFrame(runFileBuffer)) {
+ previousReadPointer = tempFrame;
+ accessor.reset(runFileBuffer.getBuffer());
+ accessor.next();
+ readCount++;
+ return true;
+ }
+ return false;
+ }
+
+ public void flushRunFile() throws HyracksDataException {
+ writing = false;
+ // Flush buffer.
+ if (runFileAppender.getTupleCount() > 0) {
+ runFileAppender.write(runFileWriter, true);
+ writeCount++;
+ }
+ runFileBuffer.reset();
+ }
+
+ public void closeRunFileReading() throws HyracksDataException {
+ reading = false;
+ runFileReader.close();
+ previousReadPointer = -1;
+ }
+
+ public void close() throws HyracksDataException {
+ if (runFileReader != null) {
+ runFileReader.close();
+ }
+ if (runFileWriter != null) {
+ runFileWriter.close();
+ }
+ }
+
+ public void removeRunFile() {
+ if (runfile != null) {
+ FileUtils.deleteQuietly(runfile.getFile());
+ }
+ }
+
+ public boolean isReading() {
+ return reading;
+ }
+
+ public boolean isWriting() {
+ return writing;
+ }
+
+ public long getReadPointer() {
+ if (runFileReader != null) {
+ return previousReadPointer;
+ }
+ return -1;
+ }
+
+ public ByteBuffer getAppenderBuffer() {
+ return runFileAppender.getBuffer();
+ }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/TupleAccessor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/TupleAccessor.java
new file mode 100644
index 0000000..b97df37
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/interval/utils/memory/TupleAccessor.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.interval.utils.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class TupleAccessor extends FrameTupleAccessor implements ITupleAccessor {
+ public static final int UNSET = -2;
+ public static final int INITIALIZED = -1;
+ private int tupleId = UNSET;
+
+ public TupleAccessor(RecordDescriptor recordDescriptor) {
+ super(recordDescriptor);
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ reset(buffer, 0, buffer.limit());
+ tupleId = INITIALIZED;
+ }
+
+ public void reset(TuplePointer tp) {
+ throw new IllegalAccessError("Should never call this reset");
+ }
+
+ @Override
+ public int getTupleStartOffset() {
+ return getTupleStartOffset(tupleId);
+ }
+
+ @Override
+ public int getTupleEndOffset() {
+ return getTupleStartOffset(tupleId);
+ }
+
+ @Override
+ public int getTupleLength() {
+ return getTupleLength(tupleId);
+ }
+
+ @Override
+ public int getAbsFieldStartOffset(int fieldId) {
+ return getAbsoluteFieldStartOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public int getFieldLength(int fieldId) {
+ return getFieldLength(tupleId, fieldId);
+ }
+
+ @Override
+ public int getFieldEndOffset(int fieldId) {
+ return getFieldEndOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public int getFieldStartOffset(int fieldId) {
+ return getFieldStartOffset(tupleId, fieldId);
+ }
+
+ @Override
+ public int getTupleId() {
+ return tupleId;
+ }
+
+ @Override
+ public void getTuplePointer(TuplePointer tp) {
+ tp.reset(INITIALIZED, tupleId);
+ }
+
+ @Override
+ public void setTupleId(int tupleId) {
+ this.tupleId = tupleId;
+ }
+
+ @Override
+ public void reset() {
+ tupleId = INITIALIZED;
+ }
+
+ @Override
+ public boolean hasNext() {
+ if (tupleId == UNSET) {
+ return false;
+ }
+ return tupleId + 1 < getTupleCount();
+ }
+
+ @Override
+ public void next() {
+ ++tupleId;
+ }
+
+ @Override
+ public boolean exists() {
+ return INITIALIZED < tupleId && tupleId < getTupleCount();
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index 84d19c1..b7f6d62 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -40,6 +40,7 @@
INDEX_BULKLOAD,
INDEX_INSERT_DELETE,
INSERT_DELETE,
+ INTERVAL_MERGE_JOIN,
LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
MATERIALIZE,
MICRO_PRE_CLUSTERED_GROUP_BY,
@@ -48,6 +49,8 @@
NESTED_LOOP,
NESTED_TUPLE_SOURCE,
ONE_TO_ONE_EXCHANGE,
+ PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE,
+ PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE,
PRE_CLUSTERED_GROUP_BY,
PRE_SORTED_DISTINCT_BY,
RANDOM_PARTITION_EXCHANGE,
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 e20e534..182f61d 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
@@ -73,9 +73,11 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.PartialBroadcastRangeFollowingExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.PartialBroadcastRangeIntersectExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -143,6 +145,7 @@
case ONE_TO_ONE_EXCHANGE:
case RANDOM_MERGE_EXCHANGE:
case SEQUENTIAL_MERGE_EXCHANGE:
+ case RANDOM_PARTITION_EXCHANGE:
// No variables used.
break;
case HASH_PARTITION_EXCHANGE:
@@ -163,18 +166,25 @@
}
break;
case RANGE_PARTITION_EXCHANGE:
- RangePartitionExchangePOperator rangePartitionPOp = (RangePartitionExchangePOperator) physOp;
- for (OrderColumn partCol : rangePartitionPOp.getPartitioningFields()) {
- usedVariables.add(partCol.getColumn());
+ RangePartitionExchangePOperator rangeExchangePOp = (RangePartitionExchangePOperator) physOp;
+ for (OrderColumn orderCol : rangeExchangePOp.getPartitioningFields()) {
+ usedVariables.add(orderCol.getColumn());
}
break;
- case RANGE_PARTITION_MERGE_EXCHANGE:
- RangePartitionMergeExchangePOperator rangeMergePOp = (RangePartitionMergeExchangePOperator) physOp;
- for (OrderColumn partCol : rangeMergePOp.getPartitioningFields()) {
- usedVariables.add(partCol.getColumn());
+ case PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE:
+ PartialBroadcastRangeFollowingExchangePOperator rangeFollowingPOp =
+ (PartialBroadcastRangeFollowingExchangePOperator) physOp;
+ for (OrderColumn orderCol : rangeFollowingPOp.getPartitioningFields()) {
+ usedVariables.add(orderCol.getColumn());
}
break;
- case RANDOM_PARTITION_EXCHANGE:
+ case PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE:
+ PartialBroadcastRangeIntersectExchangePOperator intersectPOp =
+ (PartialBroadcastRangeIntersectExchangePOperator) physOp;
+ for (IntervalColumn intervalCol : intersectPOp.getIntervalFields()) {
+ usedVariables.add(intervalCol.getStartColumn());
+ usedVariables.add(intervalCol.getEndColumn());
+ }
break;
default:
throw new AlgebricksException("Unhandled physical operator tag '" + physOp.getOperatorTag() + "'.");
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractRangeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractRangeExchangePOperator.java
new file mode 100644
index 0000000..53f3aaf
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractRangeExchangePOperator.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.operators.physical;
+
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.range.DynamicRangeMapSupplier;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMapSupplier;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticRangeMapSupplier;
+
+abstract class AbstractRangeExchangePOperator extends AbstractExchangePOperator {
+
+ protected final List<OrderColumn> partitioningFields;
+
+ protected final INodeDomain domain;
+
+ protected final RangeMap rangeMap;
+
+ protected final boolean rangeMapIsComputedAtRunTime;
+
+ protected final String rangeMapKeyInContext;
+
+ private AbstractRangeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, RangeMap rangeMap,
+ boolean rangeMapIsComputedAtRunTime, String rangeMapKeyInContext) {
+ this.partitioningFields = partitioningFields;
+ this.domain = domain;
+ this.rangeMap = rangeMap;
+ this.rangeMapIsComputedAtRunTime = rangeMapIsComputedAtRunTime;
+ this.rangeMapKeyInContext = rangeMapKeyInContext;
+ }
+
+ protected AbstractRangeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+ String rangeMapKeyInContext) {
+ this(partitioningFields, domain, null, true, rangeMapKeyInContext);
+ }
+
+ protected AbstractRangeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+ RangeMap rangeMap) {
+ this(partitioningFields, domain, rangeMap, false, "");
+ }
+
+ public final List<OrderColumn> getPartitioningFields() {
+ return partitioningFields;
+ }
+
+ public final INodeDomain getDomain() {
+ return domain;
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + partitioningFields
+ + (rangeMap != null ? " RANGE_MAP:" + rangeMap : "");
+ }
+
+ protected final RangeMapSupplier crateRangeMapSupplier() {
+ return rangeMapIsComputedAtRunTime ? new DynamicRangeMapSupplier(rangeMapKeyInContext)
+ : new StaticRangeMapSupplier(rangeMap);
+ }
+
+ protected Pair<int[], IBinaryComparatorFactory[]> createOrderColumnsAndComparators(ILogicalOperator op,
+ IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int n = partitioningFields.size();
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ int i = 0;
+ for (OrderColumn oc : partitioningFields) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[i] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderOperator.IOrder.OrderKind.ASC);
+ i++;
+ }
+ return new Pair<>(sortFields, comps);
+ }
+
+ protected Triple<int[], IBinaryComparatorFactory[], INormalizedKeyComputerFactory> createOrderColumnsAndComparatorsWithNormKeyComputer(
+ ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int n = partitioningFields.size();
+ int[] sortFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
+ INormalizedKeyComputerFactory nkcf = null;
+ int i = 0;
+ for (OrderColumn oc : partitioningFields) {
+ LogicalVariable var = oc.getColumn();
+ sortFields[i] = opSchema.findVariable(var);
+ Object type = env.getVarType(var);
+ OrderOperator.IOrder.OrderKind order = oc.getOrder();
+ if (i == 0 && nkcfProvider != null && type != null) {
+ nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderOperator.IOrder.OrderKind.ASC);
+ }
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderOperator.IOrder.OrderKind.ASC);
+ i++;
+ }
+ return new Triple<>(sortFields, comps, nkcf);
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeFollowingExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeFollowingExchangePOperator.java
new file mode 100644
index 0000000..46b7fef
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeFollowingExchangePOperator.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.operators.physical;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+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.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedFollowingProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITupleMultiPartitionComputerFactory;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangeFollowingPartitionComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.std.connectors.MToNPartialBroadcastConnectorDescriptor;
+
+/**
+ * This exchange operator delivers {@link IPartitioningProperty.PartitioningType#PARTIAL_BROADCAST_ORDERED_FOLLOWING}
+ * structural property
+ */
+public final class PartialBroadcastRangeFollowingExchangePOperator extends AbstractRangeExchangePOperator {
+
+ public PartialBroadcastRangeFollowingExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+ RangeMap rangeMap) {
+ super(partitioningFields, domain, rangeMap);
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.PARTIAL_BROADCAST_RANGE_FOLLOWING_EXCHANGE;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ IPartitioningProperty pp = new PartialBroadcastOrderedFollowingProperty(partitioningFields, domain, rangeMap);
+ // Broadcasts destroy input's local properties.
+ this.deliveredProperties = new StructuralPropertiesVector(pp, Collections.emptyList());
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, IHyracksJobBuilder.TargetConstraint> createConnectorDescriptor(
+ IConnectorDescriptorRegistry spec, ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context)
+ throws AlgebricksException {
+ Pair<int[], IBinaryComparatorFactory[]> pOrderColumns = createOrderColumnsAndComparators(op, opSchema, context);
+ ITupleMultiPartitionComputerFactory tpcf = new FieldRangeFollowingPartitionComputerFactory(pOrderColumns.first,
+ pOrderColumns.second, crateRangeMapSupplier(), op.getSourceLocation());
+ IConnectorDescriptor conn = new MToNPartialBroadcastConnectorDescriptor(spec, tpcf);
+ return new Pair<>(conn, null);
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeIntersectExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeIntersectExchangePOperator.java
new file mode 100644
index 0000000..4097101
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/PartialBroadcastRangeIntersectExchangePOperator.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.operators.physical;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedIntersectProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ITupleMultiPartitionComputerFactory;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangeIntersectPartitionComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticRangeMapSupplier;
+import org.apache.hyracks.dataflow.std.connectors.MToNPartialBroadcastConnectorDescriptor;
+
+/**
+ * This exchange operator delivers {@link IPartitioningProperty.PartitioningType#PARTIAL_BROADCAST_ORDERED_INTERSECT}
+ * structural property
+ */
+public class PartialBroadcastRangeIntersectExchangePOperator extends AbstractExchangePOperator {
+
+ private final List<IntervalColumn> intervalFields;
+
+ private final INodeDomain domain;
+
+ private final RangeMap rangeMap;
+
+ public PartialBroadcastRangeIntersectExchangePOperator(List<IntervalColumn> intervalFields, INodeDomain domain,
+ RangeMap rangeMap) {
+ this.intervalFields = intervalFields;
+ this.domain = domain;
+ this.rangeMap = rangeMap;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.PARTIAL_BROADCAST_RANGE_INTERSECT_EXCHANGE;
+ }
+
+ public final List<IntervalColumn> getIntervalFields() {
+ return intervalFields;
+ }
+
+ public final INodeDomain getDomain() {
+ return domain;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ IPartitioningProperty pp = new PartialBroadcastOrderedIntersectProperty(intervalFields, domain, rangeMap);
+ // Broadcasts destroy input's local properties.
+ this.deliveredProperties = new StructuralPropertiesVector(pp, Collections.emptyList());
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+ return emptyUnaryRequirements();
+ }
+
+ @Override
+ public Pair<IConnectorDescriptor, IHyracksJobBuilder.TargetConstraint> createConnectorDescriptor(
+ IConnectorDescriptorRegistry spec, ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context)
+ throws AlgebricksException {
+ Triple<int[], int[], IBinaryComparatorFactory[]> pIntervalColumns =
+ createIntervalColumnsAndComparators(op, opSchema, context);
+ ITupleMultiPartitionComputerFactory tpcf =
+ new FieldRangeIntersectPartitionComputerFactory(pIntervalColumns.first, pIntervalColumns.second,
+ pIntervalColumns.third, new StaticRangeMapSupplier(rangeMap), op.getSourceLocation());
+ IConnectorDescriptor conn = new MToNPartialBroadcastConnectorDescriptor(spec, tpcf);
+ return new Pair<>(conn, null);
+ }
+
+ private Triple<int[], int[], IBinaryComparatorFactory[]> createIntervalColumnsAndComparators(ILogicalOperator op,
+ IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
+ int n = intervalFields.size();
+ int[] startFields = new int[n];
+ int[] endFields = new int[n];
+ IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ int i = 0;
+ for (IntervalColumn ic : intervalFields) {
+ LogicalVariable startVar = ic.getStartColumn();
+ startFields[i] = opSchema.findVariable(startVar);
+ Object startVarType = env.getVarType(startVar);
+ LogicalVariable endEvar = ic.getEndColumn();
+ endFields[i] = opSchema.findVariable(endEvar);
+ Object endVarType = env.getVarType(endEvar);
+ if (!Objects.equals(startVarType, endVarType)) {
+ throw new IllegalStateException();
+ }
+ IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
+ comps[i] =
+ bcfp.getBinaryComparatorFactory(startVarType, ic.getOrder() == OrderOperator.IOrder.OrderKind.ASC);
+ i++;
+ }
+ return new Triple<>(startFields, endFields, comps);
+ }
+
+ @Override
+ public String toString() {
+ return getOperatorTag().toString() + " " + intervalFields + (rangeMap != null ? " RANGE_MAP:" + rangeMap : "");
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
index fe96d4f..9c80f4c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
@@ -19,7 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.physical;
import java.util.ArrayList;
-import java.util.LinkedList;
+import java.util.Collections;
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -27,12 +27,8 @@
import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
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.properties.ILocalStructuralProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
@@ -41,42 +37,23 @@
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.common.data.partition.range.DynamicRangeMapSupplier;
import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
-import org.apache.hyracks.dataflow.common.data.partition.range.RangeMapSupplier;
-import org.apache.hyracks.dataflow.common.data.partition.range.StaticRangeMapSupplier;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
+public final class RangePartitionExchangePOperator extends AbstractRangeExchangePOperator {
- private List<OrderColumn> partitioningFields;
- private INodeDomain domain;
- private RangeMap rangeMap;
- private final boolean rangeMapIsComputedAtRunTime;
- private final String rangeMapKeyInContext;
-
- private RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, RangeMap rangeMap,
- boolean rangeMapIsComputedAtRunTime, String rangeMapKeyInContext) {
- this.partitioningFields = partitioningFields;
- this.domain = domain;
- this.rangeMap = rangeMap;
- this.rangeMapIsComputedAtRunTime = rangeMapIsComputedAtRunTime;
- this.rangeMapKeyInContext = rangeMapKeyInContext;
- }
-
- public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, String rangeMapKeyInContext,
- INodeDomain domain) {
- this(partitioningFields, domain, null, true, rangeMapKeyInContext);
+ public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+ String rangeMapKeyInContext) {
+ super(partitioningFields, domain, rangeMapKeyInContext);
}
public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
RangeMap rangeMap) {
- this(partitioningFields, domain, rangeMap, false, "");
+ super(partitioningFields, domain, rangeMap);
}
@Override
@@ -84,18 +61,11 @@
return PhysicalOperatorTag.RANGE_PARTITION_EXCHANGE;
}
- public List<OrderColumn> getPartitioningFields() {
- return partitioningFields;
- }
-
- public INodeDomain getDomain() {
- return domain;
- }
-
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
- IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<>(partitioningFields), domain, rangeMap);
- this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
+ IPartitioningProperty pp =
+ new OrderedPartitionedProperty(new ArrayList<>(partitioningFields), domain, rangeMap);
+ this.deliveredProperties = new StructuralPropertiesVector(pp, Collections.emptyList());
}
@Override
@@ -107,30 +77,10 @@
@Override
public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(IConnectorDescriptorRegistry spec,
ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
- int n = partitioningFields.size();
- int[] sortFields = new int[n];
- IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
- IVariableTypeEnvironment env = context.getTypeEnvironment(op);
- int i = 0;
- for (OrderColumn oc : partitioningFields) {
- LogicalVariable var = oc.getColumn();
- sortFields[i] = opSchema.findVariable(var);
- Object type = env.getVarType(var);
- IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
- comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
- i++;
- }
- RangeMapSupplier rangeMapSupplier = rangeMapIsComputedAtRunTime
- ? new DynamicRangeMapSupplier(rangeMapKeyInContext) : new StaticRangeMapSupplier(rangeMap);
- FieldRangePartitionComputerFactory partitionerFactory =
- new FieldRangePartitionComputerFactory(sortFields, comps, rangeMapSupplier, op.getSourceLocation());
+ Pair<int[], IBinaryComparatorFactory[]> pOrderColumns = createOrderColumnsAndComparators(op, opSchema, context);
+ FieldRangePartitionComputerFactory partitionerFactory = new FieldRangePartitionComputerFactory(
+ pOrderColumns.first, pOrderColumns.second, crateRangeMapSupplier(), op.getSourceLocation());
IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, partitionerFactory);
return new Pair<>(conn, null);
}
-
- @Override
- public String toString() {
- return getOperatorTag().toString() + " " + partitioningFields
- + (rangeMap != null ? " RANGE_MAP:" + rangeMap : "");
- }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
deleted file mode 100644
index ba4a6b7..0000000
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
+++ /dev/null
@@ -1,154 +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.operators.physical;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
-import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-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.properties.ILocalStructuralProperty;
-import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
-import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
-import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
-import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
-import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
-import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
-import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
-import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
-import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
-import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
-import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
-import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
-import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
-import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
-import org.apache.hyracks.dataflow.common.data.partition.range.StaticRangeMapSupplier;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
-
-public class RangePartitionMergeExchangePOperator extends AbstractExchangePOperator {
-
- private List<OrderColumn> partitioningFields;
- private INodeDomain domain;
- private RangeMap rangeMap;
-
- public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
- RangeMap rangeMap) {
- this.partitioningFields = partitioningFields;
- this.domain = domain;
- this.rangeMap = rangeMap;
- }
-
- @Override
- public PhysicalOperatorTag getOperatorTag() {
- return PhysicalOperatorTag.RANGE_PARTITION_MERGE_EXCHANGE;
- }
-
- public List<OrderColumn> getPartitioningFields() {
- return partitioningFields;
- }
-
- public INodeDomain getDomain() {
- return domain;
- }
-
- @Override
- public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
- IPartitioningProperty p = new OrderedPartitionedProperty(partitioningFields, domain, rangeMap);
- AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
- List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
- List<ILocalStructuralProperty> locals = new ArrayList<ILocalStructuralProperty>();
- for (ILocalStructuralProperty prop : op2Locals) {
- if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
- locals.add(prop);
- } else {
- break;
- }
- }
-
- this.deliveredProperties = new StructuralPropertiesVector(p, locals);
- }
-
- @Override
- public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
- IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
- List<ILocalStructuralProperty> orderProps = new LinkedList<ILocalStructuralProperty>();
- List<OrderColumn> columns = new ArrayList<OrderColumn>();
- for (OrderColumn oc : partitioningFields) {
- LogicalVariable var = oc.getColumn();
- columns.add(new OrderColumn(var, oc.getOrder()));
- }
- orderProps.add(new LocalOrderProperty(columns));
- StructuralPropertiesVector[] r =
- new StructuralPropertiesVector[] { new StructuralPropertiesVector(null, orderProps) };
- return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
- }
-
- @Override
- public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(IConnectorDescriptorRegistry spec,
- ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
- int n = partitioningFields.size();
- int[] sortFields = new int[n];
- IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
-
- INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
- INormalizedKeyComputerFactory nkcf = null;
-
- IVariableTypeEnvironment env = context.getTypeEnvironment(op);
- int i = 0;
- for (OrderColumn oc : partitioningFields) {
- LogicalVariable var = oc.getColumn();
- sortFields[i] = opSchema.findVariable(var);
- Object type = env.getVarType(var);
- OrderKind order = oc.getOrder();
- if (i == 0 && nkcfProvider != null && type != null) {
- nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderKind.ASC);
- }
- IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
- comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
- i++;
- }
- ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps,
- new StaticRangeMapSupplier(rangeMap), op.getSourceLocation());
- IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
- return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
- }
-
- @Override
- public String toString() {
- return getOperatorTag().toString() + " " + partitioningFields
- + (rangeMap != null ? " RANGE_MAP:" + rangeMap : "");
- }
-
-}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/AbstractOrderedPartitionedProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/AbstractOrderedPartitionedProperty.java
new file mode 100644
index 0000000..7c280f4
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/AbstractOrderedPartitionedProperty.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.properties;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+abstract class AbstractOrderedPartitionedProperty implements IPartitioningProperty {
+
+ protected final List<OrderColumn> orderColumns;
+
+ protected final RangeMap rangeMap;
+
+ protected INodeDomain domain;
+
+ AbstractOrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain) {
+ this(orderColumns, domain, null);
+ }
+
+ AbstractOrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, RangeMap rangeMap) {
+ this.orderColumns = orderColumns;
+ this.domain = domain;
+ this.rangeMap = rangeMap;
+ }
+
+ public List<OrderColumn> getOrderColumns() {
+ return orderColumns;
+ }
+
+ public List<LogicalVariable> getColumns() {
+ ArrayList<LogicalVariable> cols = new ArrayList<>(orderColumns.size());
+ for (OrderColumn oc : orderColumns) {
+ cols.add(oc.getColumn());
+ }
+ return cols;
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType().toString() + orderColumns + " domain:" + domain
+ + (rangeMap != null ? " range-map:" + rangeMap : "");
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ for (OrderColumn oc : orderColumns) {
+ columns.add(oc.getColumn());
+ }
+ }
+
+ public RangeMap getRangeMap() {
+ return rangeMap;
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+ @Override
+ public IPartitioningProperty substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
+ boolean applied = false;
+ List<OrderColumn> newOrderColumns = new ArrayList<>(orderColumns.size());
+ for (OrderColumn orderColumn : orderColumns) {
+ LogicalVariable columnVar = orderColumn.getColumn();
+ LogicalVariable newColumnVar = varMap.get(columnVar);
+ if (newColumnVar != null) {
+ applied = true;
+ } else {
+ newColumnVar = columnVar;
+ }
+ newOrderColumns.add(new OrderColumn(newColumnVar, orderColumn.getOrder()));
+ }
+ return applied ? newInstance(newOrderColumns, domain, rangeMap) : this;
+ }
+
+ @Override
+ public IPartitioningProperty clonePartitioningProperty() {
+ return newInstance(new ArrayList<>(orderColumns), domain, rangeMap);
+ }
+
+ protected abstract AbstractOrderedPartitionedProperty newInstance(List<OrderColumn> columns, INodeDomain domain,
+ RangeMap rangeMap);
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
index f5eef1c..8e181c0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
@@ -50,7 +50,27 @@
* Data is range partitioned (only used on data that has a total order).
* The partitions are order based on the data range.
*/
- ORDERED_PARTITIONED
+ ORDERED_PARTITIONED,
+ /**
+ * Data is partially replicated as follows:
+ * <ol>
+ * <li>partition the data according to the {@link #ORDERED_PARTITIONED} layout
+ * <li>then replicate each tuple in each partition to those partitions that correspond to higher ranges
+ * than the range of this tuple's original partition
+ * </ol>
+ */
+ PARTIAL_BROADCAST_ORDERED_FOLLOWING,
+ /**
+ * Data is partially replicated as follows:
+ * <ol>
+ * <li>assume there are two columns "start" and "end"
+ * <li>partition the data according to the {@link #ORDERED_PARTITIONED} layout on the "start" column
+ * <li>then replicate each tuple in each partition to those partitions that correspond to higher ranges
+ * than the range of this tuple's original partition, excluding those partitions that start at a range
+ * higher than the "end" column
+ * </ol>
+ */
+ PARTIAL_BROADCAST_ORDERED_INTERSECT
}
INodeDomain DOMAIN_FOR_UNPARTITIONED_DATA = new INodeDomain() {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IntervalColumn.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IntervalColumn.java
new file mode 100644
index 0000000..484561c
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IntervalColumn.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.hyracks.algebricks.core.algebra.properties;
+
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+
+public final class IntervalColumn {
+
+ private LogicalVariable startColumn;
+
+ private LogicalVariable endColumn;
+
+ private OrderKind order;
+
+ public IntervalColumn(LogicalVariable startColumn, LogicalVariable endColumn, OrderKind order) {
+ this.startColumn = startColumn;
+ this.endColumn = endColumn;
+ this.order = order;
+ }
+
+ public LogicalVariable getStartColumn() {
+ return startColumn;
+ }
+
+ public LogicalVariable getEndColumn() {
+ return endColumn;
+ }
+
+ public OrderKind getOrder() {
+ return order;
+ }
+
+ public void setStartColumn(LogicalVariable column) {
+ this.startColumn = column;
+ }
+
+ public void setEndColumn(LogicalVariable column) {
+ this.endColumn = column;
+ }
+
+ public void setOrder(OrderKind order) {
+ this.order = order;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (!(obj instanceof IntervalColumn)) {
+ return false;
+ } else {
+ IntervalColumn ic = (IntervalColumn) obj;
+ return startColumn.equals(ic.getStartColumn()) && endColumn.equals(ic.getEndColumn())
+ && order == ic.getOrder();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "{" + startColumn + "," + endColumn + "," + order + "}";
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
index 0136996..d4fcd36 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
@@ -18,8 +18,6 @@
*/
package org.apache.hyracks.algebricks.core.algebra.properties;
-import java.util.ArrayList;
-import java.util.Collection;
import java.util.List;
import java.util.Map;
@@ -27,32 +25,14 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
-public class OrderedPartitionedProperty implements IPartitioningProperty {
-
- private final List<OrderColumn> orderColumns;
- private final RangeMap rangeMap;
- private INodeDomain domain;
+public class OrderedPartitionedProperty extends AbstractOrderedPartitionedProperty {
public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain) {
- this(orderColumns, domain, null);
+ super(orderColumns, domain);
}
public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, RangeMap rangeMap) {
- this.domain = domain;
- this.orderColumns = orderColumns;
- this.rangeMap = rangeMap;
- }
-
- public List<OrderColumn> getOrderColumns() {
- return orderColumns;
- }
-
- public List<LogicalVariable> getColumns() {
- ArrayList<LogicalVariable> cols = new ArrayList<>(orderColumns.size());
- for (OrderColumn oc : orderColumns) {
- cols.add(oc.getColumn());
- }
- return cols;
+ super(orderColumns, domain, rangeMap);
}
@Override
@@ -61,59 +41,16 @@
}
@Override
- public String toString() {
- return getPartitioningType().toString() + orderColumns + " domain:" + domain
- + (rangeMap != null ? " range-map:" + rangeMap : "");
- }
-
- @Override
public IPartitioningProperty normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
List<FunctionalDependency> fds) {
List<OrderColumn> columns = PropertiesUtil.replaceOrderColumnsByEqClasses(orderColumns, equivalenceClasses);
columns = PropertiesUtil.applyFDsToOrderColumns(columns, fds);
- return new OrderedPartitionedProperty(columns, domain, rangeMap);
+ return newInstance(columns, domain, rangeMap);
}
@Override
- public void getColumns(Collection<LogicalVariable> columns) {
- for (OrderColumn oc : orderColumns) {
- columns.add(oc.getColumn());
- }
- }
-
- public RangeMap getRangeMap() {
- return rangeMap;
- }
-
- @Override
- public INodeDomain getNodeDomain() {
- return domain;
- }
-
- @Override
- public void setNodeDomain(INodeDomain domain) {
- this.domain = domain;
- }
-
- @Override
- public IPartitioningProperty substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
- boolean applied = false;
- List<OrderColumn> newOrderColumns = new ArrayList<>(orderColumns.size());
- for (OrderColumn orderColumn : orderColumns) {
- LogicalVariable columnVar = orderColumn.getColumn();
- LogicalVariable newColumnVar = varMap.get(columnVar);
- if (newColumnVar != null) {
- applied = true;
- } else {
- newColumnVar = columnVar;
- }
- newOrderColumns.add(new OrderColumn(newColumnVar, orderColumn.getOrder()));
- }
- return applied ? new OrderedPartitionedProperty(newOrderColumns, domain, rangeMap) : this;
- }
-
- @Override
- public IPartitioningProperty clonePartitioningProperty() {
- return new OrderedPartitionedProperty(new ArrayList<>(orderColumns), domain, rangeMap);
+ protected AbstractOrderedPartitionedProperty newInstance(List<OrderColumn> orderColumns, INodeDomain domain,
+ RangeMap rangeMap) {
+ return new OrderedPartitionedProperty(orderColumns, domain, rangeMap);
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedFollowingProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedFollowingProperty.java
new file mode 100644
index 0000000..1845d1b
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedFollowingProperty.java
@@ -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.
+ */
+
+package org.apache.hyracks.algebricks.core.algebra.properties;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public final class PartialBroadcastOrderedFollowingProperty extends AbstractOrderedPartitionedProperty {
+
+ public PartialBroadcastOrderedFollowingProperty(List<OrderColumn> orderColumns, INodeDomain domain,
+ RangeMap rangeMap) {
+ super(orderColumns, domain, rangeMap);
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.PARTIAL_BROADCAST_ORDERED_FOLLOWING;
+ }
+
+ @Override
+ public IPartitioningProperty normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
+ return this;
+ }
+
+ @Override
+ protected AbstractOrderedPartitionedProperty newInstance(List<OrderColumn> orderColumns, INodeDomain domain,
+ RangeMap rangeMap) {
+ return new PartialBroadcastOrderedFollowingProperty(orderColumns, domain, rangeMap);
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedIntersectProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedIntersectProperty.java
new file mode 100644
index 0000000..cb5b645
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PartialBroadcastOrderedIntersectProperty.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.properties;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public final class PartialBroadcastOrderedIntersectProperty implements IPartitioningProperty {
+
+ private final List<IntervalColumn> intervalColumns;
+
+ private final RangeMap rangeMap;
+
+ private INodeDomain domain;
+
+ public PartialBroadcastOrderedIntersectProperty(List<IntervalColumn> intervalColumns, INodeDomain domain,
+ RangeMap rangeMap) {
+ this.intervalColumns = intervalColumns;
+ this.domain = domain;
+ this.rangeMap = rangeMap;
+ }
+
+ @Override
+ public PartitioningType getPartitioningType() {
+ return PartitioningType.PARTIAL_BROADCAST_ORDERED_INTERSECT;
+ }
+
+ @Override
+ public String toString() {
+ return getPartitioningType().toString() + intervalColumns + " domain:" + domain
+ + (rangeMap != null ? " range-map:" + rangeMap : "");
+ }
+
+ public List<IntervalColumn> getIntervalColumns() {
+ return intervalColumns;
+ }
+
+ @Override
+ public void getColumns(Collection<LogicalVariable> columns) {
+ for (IntervalColumn ic : intervalColumns) {
+ columns.add(ic.getStartColumn());
+ columns.add(ic.getEndColumn());
+ }
+ }
+
+ public RangeMap getRangeMap() {
+ return rangeMap;
+ }
+
+ @Override
+ public INodeDomain getNodeDomain() {
+ return domain;
+ }
+
+ @Override
+ public void setNodeDomain(INodeDomain domain) {
+ this.domain = domain;
+ }
+
+ @Override
+ public IPartitioningProperty normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
+ return this;
+ }
+
+ @Override
+ public IPartitioningProperty substituteColumnVars(Map<LogicalVariable, LogicalVariable> varMap) {
+ boolean applied = false;
+ List<IntervalColumn> newIntervalColumns = new ArrayList<>(intervalColumns.size());
+ for (IntervalColumn intervalColumn : intervalColumns) {
+ LogicalVariable startColumnVar = intervalColumn.getStartColumn();
+ LogicalVariable newStartColumnVar = varMap.get(startColumnVar);
+ if (newStartColumnVar != null) {
+ applied = true;
+ } else {
+ newStartColumnVar = startColumnVar;
+ }
+
+ LogicalVariable endColumnVar = intervalColumn.getEndColumn();
+ LogicalVariable newEndColumnVar = varMap.get(endColumnVar);
+ if (newEndColumnVar != null) {
+ applied = true;
+ } else {
+ newEndColumnVar = endColumnVar;
+ }
+
+ newIntervalColumns.add(new IntervalColumn(newStartColumnVar, newEndColumnVar, intervalColumn.getOrder()));
+ }
+ return applied ? new PartialBroadcastOrderedIntersectProperty(newIntervalColumns, domain, rangeMap) : this;
+ }
+
+ @Override
+ public IPartitioningProperty clonePartitioningProperty() {
+ return new PartialBroadcastOrderedIntersectProperty(new ArrayList<>(intervalColumns), domain, rangeMap);
+ }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
index 1ebaeeb..1025e44 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
@@ -181,6 +181,32 @@
}
}
}
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING: {
+ switch (dlvd.getPartitioningType()) {
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING: {
+ PartialBroadcastOrderedFollowingProperty pr = (PartialBroadcastOrderedFollowingProperty) reqd;
+ PartialBroadcastOrderedFollowingProperty pd = (PartialBroadcastOrderedFollowingProperty) dlvd;
+ return pr.getOrderColumns().equals(pd.getOrderColumns())
+ && pr.getRangeMap().equals(pd.getRangeMap());
+ }
+ default: {
+ return false;
+ }
+ }
+ }
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT: {
+ switch (dlvd.getPartitioningType()) {
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT: {
+ PartialBroadcastOrderedIntersectProperty pr = (PartialBroadcastOrderedIntersectProperty) reqd;
+ PartialBroadcastOrderedIntersectProperty pd = (PartialBroadcastOrderedIntersectProperty) dlvd;
+ return pr.getIntervalColumns().equals(pd.getIntervalColumns())
+ && pr.getRangeMap().equals(pd.getRangeMap());
+ }
+ default: {
+ return false;
+ }
+ }
+ }
default: {
return (dlvd.getPartitioningType() == reqd.getPartitioningType());
}
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 a1ae42b..5a8c128 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
@@ -754,6 +754,12 @@
break;
case UNPARTITIONED:
stringBuilder.append("Data is in one place.");
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING:
+ stringBuilder.append("Data is partially broadcasted to partitions.");
+ break;
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT:
+ stringBuilder.append("Data is partially broadcasted to partitions.");
+ break;
}
if (nodeDomain instanceof DefaultNodeGroupDomain) {
DefaultNodeGroupDomain nd = (DefaultNodeGroupDomain) nodeDomain;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 706028b..4205dd2 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -70,6 +70,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.MicroStableSortPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.PartialBroadcastRangeFollowingExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.PartialBroadcastRangeIntersectExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
@@ -90,6 +92,8 @@
import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedFollowingProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PartialBroadcastOrderedIntersectProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.algebra.properties.PropertiesUtil;
import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
@@ -601,6 +605,18 @@
pop = new RandomPartitionExchangePOperator(nd);
break;
}
+ case PARTIAL_BROADCAST_ORDERED_FOLLOWING: {
+ PartialBroadcastOrderedFollowingProperty pbp = (PartialBroadcastOrderedFollowingProperty) pp;
+ pop = new PartialBroadcastRangeFollowingExchangePOperator(pbp.getOrderColumns(),
+ pbp.getNodeDomain(), pbp.getRangeMap());
+ break;
+ }
+ case PARTIAL_BROADCAST_ORDERED_INTERSECT: {
+ PartialBroadcastOrderedIntersectProperty pbp = (PartialBroadcastOrderedIntersectProperty) pp;
+ pop = new PartialBroadcastRangeIntersectExchangePOperator(pbp.getIntervalColumns(),
+ pbp.getNodeDomain(), pbp.getRangeMap());
+ break;
+ }
default: {
throw new NotImplementedException("Enforcer for " + pp.getPartitioningType()
+ " partitioning type has not been implemented.");
@@ -678,14 +694,16 @@
private IPhysicalOperator createRangePartitionerConnector(AbstractLogicalOperator parentOp, INodeDomain domain,
IPartitioningProperty requiredPartitioning, int childIndex, IOptimizationContext ctx)
throws AlgebricksException {
- // options for range partitioning: 1. static range map, 2. dynamic range map computed at run time
+ // options for range partitioning: 1. Range Map from Hint computed at run time 2. static range map, 3. dynamic range map computed at run time
List<OrderColumn> partitioningColumns = ((OrderedPartitionedProperty) requiredPartitioning).getOrderColumns();
- if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
- RangeMap rangeMap = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+ RangeMap rangeMap = ((OrderedPartitionedProperty) requiredPartitioning).getRangeMap();
+ if (rangeMap != null) {
return new RangePartitionExchangePOperator(partitioningColumns, domain, rangeMap);
- } else {
- return createDynamicRangePartitionExchangePOperator(parentOp, ctx, domain, partitioningColumns, childIndex);
+ } else if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+ RangeMap map = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+ return new RangePartitionExchangePOperator(partitioningColumns, domain, map);
}
+ return createDynamicRangePartitionExchangePOperator(parentOp, ctx, domain, partitioningColumns, childIndex);
}
private IPhysicalOperator createDynamicRangePartitionExchangePOperator(AbstractLogicalOperator parentOp,
@@ -732,7 +750,7 @@
parentOp.getInputs().set(childIndex, forwardRef);
parentOp.recomputeSchema();
ctx.computeAndSetTypeEnvironmentForOperator(parentOp);
- return new RangePartitionExchangePOperator(partitioningColumns, rangeMapKey, targetDomain);
+ return new RangePartitionExchangePOperator(partitioningColumns, targetDomain, rangeMapKey);
}
private static ReplicateOperator createReplicateOperator(Mutable<ILogicalOperator> inputOperator,
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
index 33d0fda..b053cac 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
@@ -42,8 +42,8 @@
private final IFramePool pool;
private final IFrameFreeSlotPolicy policy;
private final IAppendDeletableFrameTupleAccessor accessor;
- private final ArrayList<ByteBuffer> frames;
- private final RecordDescriptor recordDescriptor;
+ protected final ArrayList<ByteBuffer> frames;
+ protected final RecordDescriptor recordDescriptor;
private int numTuples;
private int statsReOrg;