[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;