merge master
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/IntervalLocalRangeSplitterOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/IntervalLocalRangeSplitterOperator.java
new file mode 100644
index 0000000..181e0fa
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/IntervalLocalRangeSplitterOperator.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.algebra.operators;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+public class IntervalLocalRangeSplitterOperator extends AbstractExtensibleLogicalOperator {
+
+    private final List<LogicalVariable> joinKeyLogicalVars;
+
+    public IntervalLocalRangeSplitterOperator(List<LogicalVariable> joinKeyLogicalVars) {
+        this.joinKeyLogicalVars = joinKeyLogicalVars;
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public IOperatorExtension newInstance() {
+        return new IntervalLocalRangeSplitterOperator(joinKeyLogicalVars);
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return "IntervalLocalRangeSplitterOperator " + joinKeyLogicalVars;
+    }
+
+    @Override
+    public void getUsedVariables(Collection<LogicalVariable> usedVars) {
+        usedVars.addAll(joinKeyLogicalVars);
+    }
+
+    @Override
+    public void getProducedVariables(Collection<LogicalVariable> producedVars) {
+        // No produced variables.
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractIntervalJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractIntervalJoinPOperator.java
new file mode 100644
index 0000000..cc2a022
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractIntervalJoinPOperator.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+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.logical.OrderOperator.IOrder.OrderKind;
+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.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.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public abstract class AbstractIntervalJoinPOperator extends AbstractJoinPOperator {
+
+    private final List<LogicalVariable> keysLeftBranch;
+    private final List<LogicalVariable> keysRightBranch;
+    protected final IIntervalMergeJoinCheckerFactory mjcf;
+    private final RangeId leftRangeId;
+    private final RangeId rightRangeId;
+    private final IRangeMap rangeMapHint;
+
+    public AbstractIntervalJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+            List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities,
+            IIntervalMergeJoinCheckerFactory mjcf, RangeId leftRangeId, RangeId rightRangeId, IRangeMap rangeMapHint) {
+        super(kind, partitioningType);
+        this.keysLeftBranch = sideLeftOfEqualities;
+        this.keysRightBranch = sideRightOfEqualities;
+        this.mjcf = mjcf;
+        this.leftRangeId = leftRangeId;
+        this.rightRangeId = rightRangeId;
+        this.rangeMapHint = rangeMapHint;
+    }
+
+    public List<LogicalVariable> getKeysLeftBranch() {
+        return keysLeftBranch;
+    }
+
+    public List<LogicalVariable> getKeysRightBranch() {
+        return keysRightBranch;
+    }
+
+    public IIntervalMergeJoinCheckerFactory getIntervalMergeJoinCheckerFactory() {
+        return mjcf;
+    }
+
+    public RangeId getLeftRangeId() {
+        return leftRangeId;
+    }
+
+    public RangeId getRightRangeId() {
+        return rightRangeId;
+    }
+
+    public IRangeMap getRangeMapHint() {
+        return rangeMapHint;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.EXTENSION_OPERATOR;
+    }
+
+    @Override
+    public String toString() {
+        return getIntervalJoin() + " " + keysLeftBranch + " " + keysRightBranch;
+    }
+
+    public abstract String getIntervalJoin();
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context) {
+        ArrayList<OrderColumn> order = getLeftRangeOrderColumn();
+        IPartitioningProperty pp = new OrderedPartitionedProperty(order, null, leftRangeId,
+                RangePartitioningType.PROJECT, rangeMapHint);
+        List<ILocalStructuralProperty> propsLocal = new ArrayList<>();
+        propsLocal.add(new LocalOrderProperty(getLeftLocalSortOrderColumn()));
+        deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+        IPartitioningProperty ppLeft = null;
+        List<ILocalStructuralProperty> ispLeft = new ArrayList<>();
+        ispLeft.add(new LocalOrderProperty(getLeftLocalSortOrderColumn()));
+
+        IPartitioningProperty ppRight = null;
+        List<ILocalStructuralProperty> ispRight = new ArrayList<>();
+        ispRight.add(new LocalOrderProperty(getRightLocalSortOrderColumn()));
+
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+            ppLeft = new OrderedPartitionedProperty(getLeftRangeOrderColumn(), null, leftRangeId,
+                    mjcf.getLeftPartitioningType(), rangeMapHint);
+            ppRight = new OrderedPartitionedProperty(getRightRangeOrderColumn(), null, rightRangeId,
+                    mjcf.getRightPartitioningType(), rangeMapHint);
+        }
+
+        pv[0] = new StructuralPropertiesVector(ppLeft, ispLeft);
+        pv[1] = new StructuralPropertiesVector(ppRight, ispRight);
+        IPartitioningRequirementsCoordinator prc = IPartitioningRequirementsCoordinator.NO_COORDINATION;
+        return new PhysicalRequirements(pv, prc);
+    }
+
+    protected ArrayList<OrderColumn> getLeftLocalSortOrderColumn() {
+        return getLeftRangeOrderColumn();
+    }
+
+    protected ArrayList<OrderColumn> getRightLocalSortOrderColumn() {
+        return getRightRangeOrderColumn();
+    }
+
+    protected ArrayList<OrderColumn> getLeftRangeOrderColumn() {
+        ArrayList<OrderColumn> order = new ArrayList<>();
+        for (LogicalVariable v : keysLeftBranch) {
+            order.add(new OrderColumn(v, mjcf.isOrderAsc() ? OrderKind.ASC : OrderKind.DESC));
+        }
+        return order;
+    }
+
+    protected ArrayList<OrderColumn> getRightRangeOrderColumn() {
+        ArrayList<OrderColumn> orderRight = new ArrayList<>();
+        for (LogicalVariable v : keysRightBranch) {
+            orderRight.add(new OrderColumn(v, mjcf.isOrderAsc() ? OrderKind.ASC : OrderKind.DESC));
+        }
+        return orderRight;
+    }
+
+    @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,
+                leftRangeId);
+        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);
+    }
+
+    abstract IOperatorDescriptor getIntervalOperatorDescriptor(int[] keysLeft, int[] keysRight,
+            IOperatorDescriptorRegistry spec, RecordDescriptor recordDescriptor, IIntervalMergeJoinCheckerFactory mjcf,
+            RangeId rangeId);
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalIndexJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalIndexJoinPOperator.java
new file mode 100644
index 0000000..731b39e
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalIndexJoinPOperator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.algebra.operators.physical;
+
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.intervalindex.IntervalIndexJoinOperatorDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public class IntervalIndexJoinPOperator extends AbstractIntervalJoinPOperator {
+
+    private final int memSizeInFrames;
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalIndexJoinPOperator.class.getName());
+
+    public IntervalIndexJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+            List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities,
+            int memSizeInFrames, IIntervalMergeJoinCheckerFactory mjcf, RangeId leftRangeId, RangeId rightRangeId,
+            IRangeMap rangeMapHint) {
+        super(kind, partitioningType, sideLeftOfEqualities, sideRightOfEqualities, mjcf, leftRangeId, rightRangeId,
+                rangeMapHint);
+        this.memSizeInFrames = memSizeInFrames;
+
+        LOGGER.fine("IntervalIndexJoinPOperator constructed with: JoinKind=" + kind + ", JoinPartitioningType="
+                + partitioningType + ", List<LogicalVariable>=" + sideLeftOfEqualities + ", List<LogicalVariable>="
+                + sideRightOfEqualities + ", int memSizeInFrames=" + memSizeInFrames
+                + ", IMergeJoinCheckerFactory mjcf=" + mjcf + ", RangeId leftRangeId=" + leftRangeId
+                + ", RangeId rightRangeId=" + rightRangeId + ".");
+    }
+
+    @Override
+    public String getIntervalJoin() {
+        return "INTERVAL_INDEX_JOIN";
+    }
+
+    @Override
+    IOperatorDescriptor getIntervalOperatorDescriptor(int[] keysLeft, int[] keysRight, IOperatorDescriptorRegistry spec,
+            RecordDescriptor recordDescriptor, IIntervalMergeJoinCheckerFactory mjcf, RangeId rangeId) {
+        return new IntervalIndexJoinOperatorDescriptor(spec, memSizeInFrames, keysLeft, keysRight, recordDescriptor,
+                mjcf);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeOperatorDescriptor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeOperatorDescriptor.java
new file mode 100644
index 0000000..cf8ad89
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeOperatorDescriptor.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.nio.ByteBuffer;
+
+import org.apache.asterix.runtime.operators.joins.IntervalJoinUtil;
+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.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.value.IRangeMap;
+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.data.std.primitive.LongPointable;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+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.AbstractUnaryInputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+
+public class IntervalLocalRangeOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private static final int PARTITION_ACTIVITY_ID = 0;
+
+    private static final int OUTPUT_ARITY = 3;
+
+    private static final int INPUT_STARTS = 0;
+    private static final int INPUT_COVERS = 2;
+    private static final int INPUT_ENDS = 1;
+
+    private final int key;
+    private final RangeId rangeId;
+
+    public IntervalLocalRangeOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys,
+            RecordDescriptor recordDescriptor, RangeId rangeId) {
+        super(spec, 1, OUTPUT_ARITY);
+        for (int i = 0; i < outputArity; i++) {
+            recordDescriptors[i] = recordDescriptor;
+        }
+        key = keys[0];
+        this.rangeId = rangeId;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId aid = new ActivityId(odId, PARTITION_ACTIVITY_ID);
+        IActivity phase = new PartitionActivityNode(aid);
+
+        builder.addActivity(this, phase);
+        builder.addSourceEdge(0, phase, 0);
+        // Connect output
+        builder.addTargetEdge(0, phase, 0);
+        builder.addTargetEdge(1, phase, 1);
+        builder.addTargetEdge(2, phase, 2);
+    }
+
+    private final class PartitionActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public PartitionActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new AbstractUnaryInputOperatorNodePushable() {
+                private final IFrameWriter[] writers = new IFrameWriter[getOutputArity()];
+                private final FrameTupleAppender[] resultAppender = new FrameTupleAppender[getOutputArity()];
+                private final RecordDescriptor rd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+                private final FrameTupleAccessor accessor = new FrameTupleAccessor(rd);
+                private long nodeRangeStart;
+                private long nodeRangeEnd;
+
+                @Override
+                public void close() throws HyracksDataException {
+                    flush();
+                    for (int i = 0; i < getOutputArity(); i++) {
+                        writers[i].close();
+                    }
+                }
+
+                @Override
+                public void flush() throws HyracksDataException {
+                    for (int i = 0; i < getOutputArity(); i++) {
+                        FrameUtils.flushFrame(resultAppender[i].getBuffer(), writers[i]);
+                    }
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    for (int i = 0; i < getOutputArity(); i++) {
+                        writers[i].fail();
+                    }
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    accessor.reset(buffer);
+                    int tupleCount = accessor.getTupleCount();
+                    for (int i = 0; i < tupleCount; i++) {
+                        int pid = localPartition(accessor, i, key);
+                        if (pid < outputArity) {
+                            FrameUtils.appendToWriter(writers[pid], resultAppender[pid], accessor, i);
+                        }
+                    }
+                }
+
+                private int localPartition(FrameTupleAccessor accessor, int i, int key) {
+                    long start = IntervalJoinUtil.getIntervalStart(accessor, i, key);
+                    if (start < nodeRangeStart) {
+                        long end = IntervalJoinUtil.getIntervalEnd(accessor, i, key);
+                        if (end < nodeRangeEnd) {
+                            // Ends
+                            return INPUT_ENDS;
+                        } else {
+                            // Covers (match will all intervals)
+                            return INPUT_COVERS;
+                        }
+                    } else {
+                        // Start (responsible for matches)
+                        return INPUT_STARTS;
+                    }
+                }
+
+                @Override
+                public void open() throws HyracksDataException {
+                    for (int i = 0; i < getOutputArity(); i++) {
+                        writers[i].open();
+                        resultAppender[i] = new FrameTupleAppender(new VSizeFrame(ctx), true);
+                    }
+                    RangeForwardTaskState rangeState = RangeForwardTaskState.getRangeState(rangeId.getId(), ctx);
+                    IRangeMap rangeMap = rangeState.getRangeMap();
+                    nodeRangeStart = getPartitionBoundryStart(rangeMap);
+                    nodeRangeEnd = getPartitionBoundryEnd(rangeMap);
+
+                }
+
+                @Override
+                public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+                    writers[index] = writer;
+                }
+
+                long getPartitionBoundryStart(IRangeMap rangeMap) {
+                    int fieldIndex = 0;
+                    int slot = partition - 1;
+                    long boundary = Long.MIN_VALUE;
+                    // All lookups are on typed values.
+                    if (partition == 0) {
+                        boundary = LongPointable.getLong(rangeMap.getMinByteArray(fieldIndex),
+                                rangeMap.getMinStartOffset(fieldIndex) + 1);
+                    } else if (partition <= rangeMap.getSplitCount()) {
+                        boundary = LongPointable.getLong(rangeMap.getByteArray(fieldIndex, slot),
+                                rangeMap.getStartOffset(fieldIndex, slot) + 1);
+                    } else if (partition > rangeMap.getSplitCount()) {
+                        boundary = LongPointable.getLong(rangeMap.getMaxByteArray(fieldIndex),
+                                rangeMap.getMaxStartOffset(fieldIndex) + 1);
+                    }
+                    return boundary;
+                }
+
+                long getPartitionBoundryEnd(IRangeMap rangeMap) {
+                    int fieldIndex = 0;
+                    int slot = partition;
+                    long boundary = Long.MAX_VALUE;
+                    // All lookups are on typed values.
+                    if (partition < rangeMap.getSplitCount()) {
+                        boundary = LongPointable.getLong(rangeMap.getByteArray(fieldIndex, slot),
+                                rangeMap.getStartOffset(fieldIndex, slot) + 1);
+                    } else if (partition == rangeMap.getSplitCount()) {
+                        boundary = LongPointable.getLong(rangeMap.getMaxByteArray(fieldIndex),
+                                rangeMap.getMaxStartOffset(fieldIndex) + 1);
+                    }
+                    return boundary;
+                }
+            };
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeSplitterPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeSplitterPOperator.java
new file mode 100644
index 0000000..fe5c28b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalLocalRangeSplitterPOperator.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+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.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+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.std.base.RangeId;
+
+public class IntervalLocalRangeSplitterPOperator extends AbstractPhysicalOperator {
+
+    private List<LogicalVariable> intervalFields;
+    private RangeId rangeId;
+
+    public IntervalLocalRangeSplitterPOperator(List<LogicalVariable> intervalFields, RangeId rangeId) {
+        this.intervalFields = intervalFields;
+        this.rangeId = rangeId;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.EXTENSION_OPERATOR;
+    }
+
+    @Override
+    public String toString() {
+        return "IntervalLocalRangeSplitterPOperator " + intervalFields;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        IOperatorDescriptorRegistry spec = builder.getJobSpec();
+        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                propagatedSchema, context);
+
+        int[] keys = JobGenHelper.variablesToFieldIndexes(intervalFields, inputSchemas[0]);
+
+        IOperatorDescriptor opDesc = new IntervalLocalRangeOperatorDescriptor(spec, keys, recDescriptor, rangeId);
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+
+    @Override
+    public Pair<int[], int[]> getInputOutputDependencyLabels(ILogicalOperator op) {
+        ReplicateOperator rop = (ReplicateOperator) op;
+        int[] inputDependencyLabels = new int[] { 0 };
+        int[] outputDependencyLabels = new int[rop.getOutputArity()]; // filled with 0's
+        return new Pair<>(inputDependencyLabels, outputDependencyLabels);
+    }
+
+    @Override
+    public boolean expensiveThanMaterialization() {
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalPartitionJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalPartitionJoinPOperator.java
new file mode 100644
index 0000000..73d159e
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/IntervalPartitionJoinPOperator.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.asterix.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.intervalpartition.IntervalPartitionJoinOperatorDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public class IntervalPartitionJoinPOperator extends AbstractIntervalJoinPOperator {
+    private static final int START = 0;
+    private static final int END = 1;
+
+    private final int memSizeInFrames;
+    private final int k;
+    private final List<LogicalVariable> leftPartitionVar;
+    private final List<LogicalVariable> rightPartitionVar;
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalPartitionJoinPOperator.class.getName());
+
+    public IntervalPartitionJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+            List<LogicalVariable> sideLeftOfEqualities, List<LogicalVariable> sideRightOfEqualities,
+            int memSizeInFrames, int k, IIntervalMergeJoinCheckerFactory mjcf, List<LogicalVariable> leftPartitionVar,
+            List<LogicalVariable> rightPartitionVar, RangeId leftRangeId, RangeId rightRangeId,
+            IRangeMap rangeMapHint) {
+        super(kind, partitioningType, sideLeftOfEqualities, sideRightOfEqualities, mjcf, leftRangeId, rightRangeId,
+                rangeMapHint);
+        this.memSizeInFrames = memSizeInFrames;
+        this.k = k;
+        this.leftPartitionVar = leftPartitionVar;
+        this.rightPartitionVar = rightPartitionVar;
+
+        LOGGER.fine("IntervalPartitionJoinPOperator constructed with: JoinKind=" + kind + ", JoinPartitioningType="
+                + partitioningType + ", List<LogicalVariable>=" + sideLeftOfEqualities + ", List<LogicalVariable>="
+                + sideRightOfEqualities + ", int memSizeInFrames=" + memSizeInFrames + ", int k=" + k
+                + ", IMergeJoinCheckerFactory mjcf=" + mjcf + ", RangeId leftRangeId=" + leftRangeId
+                + ", RangeId rightRangeId=" + rightRangeId + ".");
+    }
+
+    public int getK() {
+        return k;
+    }
+
+    public List<LogicalVariable> getLeftPartitionVar() {
+        return leftPartitionVar;
+    }
+
+    public List<LogicalVariable> getRightPartitionVar() {
+        return rightPartitionVar;
+    }
+
+    @Override
+    public String getIntervalJoin() {
+        return "INTERVAL_PARTITION_JOIN";
+    }
+
+    @Override
+    IOperatorDescriptor getIntervalOperatorDescriptor(int[] keysLeft, int[] keysRight, IOperatorDescriptorRegistry spec,
+            RecordDescriptor recordDescriptor, IIntervalMergeJoinCheckerFactory mjcf, RangeId rangeId) {
+        return new IntervalPartitionJoinOperatorDescriptor(spec, memSizeInFrames, k, keysLeft, keysRight,
+                recordDescriptor, mjcf, rangeId);
+    }
+
+    @Override
+    protected ArrayList<OrderColumn> getLeftLocalSortOrderColumn() {
+        ArrayList<OrderColumn> order = new ArrayList<>();
+        if (mjcf.isOrderAsc()) {
+            order.add(new OrderColumn(leftPartitionVar.get(END), OrderKind.ASC));
+            order.add(new OrderColumn(leftPartitionVar.get(START), OrderKind.DESC));
+        } else {
+            // TODO What does Desc'ing mean?
+            order.add(new OrderColumn(leftPartitionVar.get(START), OrderKind.ASC));
+            order.add(new OrderColumn(leftPartitionVar.get(END), OrderKind.DESC));
+        }
+        return order;
+    }
+
+    @Override
+    protected ArrayList<OrderColumn> getRightLocalSortOrderColumn() {
+        ArrayList<OrderColumn> order = new ArrayList<>();
+        if (mjcf.isOrderAsc()) {
+            order.add(new OrderColumn(rightPartitionVar.get(END), OrderKind.ASC));
+            order.add(new OrderColumn(rightPartitionVar.get(START), OrderKind.DESC));
+        } else {
+            // TODO What does Desc'ing mean?
+            order.add(new OrderColumn(rightPartitionVar.get(START), OrderKind.ASC));
+            order.add(new OrderColumn(rightPartitionVar.get(END), OrderKind.DESC));
+        }
+        return order;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index cd8d747..e3a9dc2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -40,6 +40,7 @@
 import org.apache.asterix.optimizer.rules.InjectTypeCastForSwitchCaseRule;
 import org.apache.asterix.optimizer.rules.InjectTypeCastForUnionRule;
 import org.apache.asterix.optimizer.rules.InlineUnnestFunctionRule;
+import org.apache.asterix.optimizer.rules.IntervalSplitPartitioningRule;
 import org.apache.asterix.optimizer.rules.IntroduceAutogenerateIDRule;
 import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastForExternalFunctionRule;
 import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
@@ -318,8 +319,8 @@
         physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
         //Turned off the following rule for now not to change OptimizerTest results.
         physicalRewritesAllLevels.add(new ReplaceSinkOpWithCommitOpRule());
-        physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
+        physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
         physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
@@ -356,6 +357,7 @@
         prepareForJobGenRewrites
                 .add(new IsolateHyracksOperatorsRule(HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
+        //prepareForJobGenRewrites.add(new IntervalSplitPartitioningRule());
         // Re-infer all types, so that, e.g., the effect of not-is-null is
         // propagated.
         prepareForJobGenRewrites.add(new ReinferAllTypesRule());
@@ -364,4 +366,5 @@
         prepareForJobGenRewrites.add(new SweepIllegalNonfunctionalFunctions());
         return prepareForJobGenRewrites;
     }
+
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntervalSplitPartitioningRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntervalSplitPartitioningRule.java
new file mode 100644
index 0000000..595b994
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntervalSplitPartitioningRule.java
@@ -0,0 +1,503 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.algebra.operators.physical.IntervalIndexJoinPOperator;
+import org.apache.asterix.algebra.operators.physical.IntervalLocalRangeSplitterPOperator;
+import org.apache.asterix.algebra.operators.physical.IntervalPartitionJoinPOperator;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+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.common.utils.ListSet;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.MaterializePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.MergeJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.NestedLoopJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnionAllPOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+/**
+ * Before:
+ *
+ * <pre>
+ *
+ * Left
+ *
+ *
+ * Right
+ * </pre>
+ *
+ * After:
+ *
+ * <pre>
+ *
+ * Left
+ *
+ *
+ * Right
+ * </pre>
+ */
+public class IntervalSplitPartitioningRule implements IAlgebraicRewriteRule {
+
+    private static final int LEFT = 0;
+    private static final int RIGHT = 1;
+
+    private static final int START_SPLITS = 3;
+
+    private static final Set<FunctionIdentifier> INTERVAL_JOIN_CONDITIONS = new HashSet<>();
+    static {
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_AFTER);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_BEFORE);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_COVERED_BY);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_COVERS);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_ENDED_BY);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_ENDS);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_MEETS);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_MET_BY);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_OVERLAPPING);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_OVERLAPS);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_STARTED_BY);
+        INTERVAL_JOIN_CONDITIONS.add(AsterixBuiltinFunctions.INTERVAL_STARTS);
+    }
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator op = opRef.getValue();
+        if (context.checkIfInDontApplySet(this, op)) {
+            return false;
+        }
+        if (!isIntervalJoin(op)) {
+            return false;
+        }
+        InnerJoinOperator originalIntervalJoin = (InnerJoinOperator) op;
+        ExecutionMode mode = originalIntervalJoin.getExecutionMode();
+        Set<LogicalVariable> localLiveVars = new ListSet<>();
+        VariableUtilities.getLiveVariables(originalIntervalJoin, localLiveVars);
+
+        Mutable<ILogicalOperator> leftSortedInput = originalIntervalJoin.getInputs().get(0);
+        Mutable<ILogicalOperator> rightSortedInput = originalIntervalJoin.getInputs().get(1);
+        if (leftSortedInput.getValue().getOperatorTag() != LogicalOperatorTag.EXCHANGE
+                && rightSortedInput.getValue().getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+            return false;
+        }
+
+        Mutable<ILogicalOperator> leftSorter = leftSortedInput.getValue().getInputs().get(0);
+        Mutable<ILogicalOperator> rightSorter = rightSortedInput.getValue().getInputs().get(0);
+        if (leftSorter.getValue().getOperatorTag() != LogicalOperatorTag.ORDER
+                && rightSorter.getValue().getOperatorTag() != LogicalOperatorTag.ORDER) {
+            return false;
+        }
+        LogicalVariable leftSortKey = getSortKey(leftSorter.getValue());
+        LogicalVariable rightSortKey = getSortKey(rightSorter.getValue());
+        if (leftSortKey == null || rightSortKey == null) {
+            return false;
+        }
+
+        Mutable<ILogicalOperator> leftRangeInput = leftSorter.getValue().getInputs().get(0);
+        Mutable<ILogicalOperator> rightRangeInput = rightSorter.getValue().getInputs().get(0);
+        RangeId leftRangeId = getRangeMapForBranch(leftRangeInput.getValue());
+        RangeId rightRangeId = getRangeMapForBranch(rightRangeInput.getValue());
+        if (leftRangeId == null || rightRangeId == null) {
+            return false;
+        }
+        // TODO check physical join
+
+        // Interval local partition operators
+        LogicalVariable leftJoinKey = getJoinKey(originalIntervalJoin.getCondition().getValue(), LEFT);
+        LogicalVariable rightJoinKey = getJoinKey(originalIntervalJoin.getCondition().getValue(), RIGHT);
+        if (leftJoinKey == null || rightJoinKey == null) {
+            return false;
+        }
+        ReplicateOperator leftIntervalSplit = getIntervalSplitOperator(leftSortKey, leftRangeId, mode);
+        Mutable<ILogicalOperator> leftIntervalSplitRef = new MutableObject<>(leftIntervalSplit);
+        ReplicateOperator rightIntervalSplit = getIntervalSplitOperator(rightSortKey, rightRangeId, mode);
+        Mutable<ILogicalOperator> rightIntervalSplitRef = new MutableObject<>(rightIntervalSplit);
+
+        // Replicate operators
+        ReplicateOperator leftStartsSplit = getReplicateOperator(START_SPLITS, mode);
+        Mutable<ILogicalOperator> leftStartsSplitRef = new MutableObject<>(leftStartsSplit);
+        ReplicateOperator rightStartsSplit = getReplicateOperator(START_SPLITS, mode);
+        Mutable<ILogicalOperator> rightStartsSplitRef = new MutableObject<>(rightStartsSplit);
+
+        // Covers Join Operator
+        ILogicalOperator leftCoversJoin = getNestedLoop(originalIntervalJoin.getCondition(), context, mode);
+        Mutable<ILogicalOperator> leftCoversJoinRef = new MutableObject<>(leftCoversJoin);
+        ILogicalOperator rightCoversJoin = getNestedLoop(originalIntervalJoin.getCondition(), context, mode);
+        Mutable<ILogicalOperator> rightCoversJoinRef = new MutableObject<>(rightCoversJoin);
+
+        // Ends Join Operator
+        ILogicalOperator startsJoin = getIntervalJoin(originalIntervalJoin, context, mode);
+        ILogicalOperator leftEndsJoin = getIntervalJoin(originalIntervalJoin, context, mode);
+        ILogicalOperator rightEndsJoin = getIntervalJoin(originalIntervalJoin, context, mode);
+        if (startsJoin == null || leftEndsJoin == null || rightEndsJoin == null) {
+            return false;
+        }
+        Mutable<ILogicalOperator> startsJoinRef = new MutableObject<>(startsJoin);
+        Mutable<ILogicalOperator> leftEndsJoinRef = new MutableObject<>(leftEndsJoin);
+        Mutable<ILogicalOperator> rightEndsJoinRef = new MutableObject<>(rightEndsJoin);
+
+        // Materialize Operator
+        ILogicalOperator leftMaterialize0 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> leftMaterialize0Ref = new MutableObject<>(leftMaterialize0);
+        ILogicalOperator leftMaterialize1 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> leftMaterialize1Ref = new MutableObject<>(leftMaterialize1);
+        ILogicalOperator leftMaterialize2 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> leftMaterialize2Ref = new MutableObject<>(leftMaterialize2);
+        ILogicalOperator rightMaterialize0 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> rightMaterialize0Ref = new MutableObject<>(rightMaterialize0);
+        ILogicalOperator rightMaterialize1 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> rightMaterialize1Ref = new MutableObject<>(rightMaterialize1);
+        ILogicalOperator rightMaterialize2 = getMaterializeOperator(mode);
+        Mutable<ILogicalOperator> rightMaterialize2Ref = new MutableObject<>(rightMaterialize2);
+
+        // Union All Operator
+        ILogicalOperator union1 = getUnionOperator(localLiveVars, mode);
+        Mutable<ILogicalOperator> union1Ref = new MutableObject<>(union1);
+        ILogicalOperator union2 = getUnionOperator(localLiveVars, mode);
+        Mutable<ILogicalOperator> union2Ref = new MutableObject<>(union2);
+        ILogicalOperator union3 = getUnionOperator(localLiveVars, mode);
+        Mutable<ILogicalOperator> union3Ref = new MutableObject<>(union3);
+        ILogicalOperator union4 = getUnionOperator(localLiveVars, mode);
+        Mutable<ILogicalOperator> union4Ref = new MutableObject<>(union4);
+
+        // Remove old path
+        originalIntervalJoin.getInputs().clear();
+
+        // Connect main path
+        connectOperators(leftIntervalSplitRef, leftSortedInput, context);
+        updateOperatorContext(context, leftIntervalSplitRef);
+        connectOperators(leftMaterialize0Ref, leftIntervalSplitRef, context);
+        updateOperatorContext(context, leftMaterialize0Ref);
+        connectOperators(leftMaterialize1Ref, leftIntervalSplitRef, context);
+        updateOperatorContext(context, leftMaterialize1Ref);
+        connectOperators(leftMaterialize2Ref, leftIntervalSplitRef, context);
+        updateOperatorContext(context, leftMaterialize2Ref);
+
+        connectOperators(leftStartsSplitRef, leftMaterialize0Ref, context);
+        updateOperatorContext(context, leftStartsSplitRef);
+
+        connectOperators(rightIntervalSplitRef, rightSortedInput, context);
+        updateOperatorContext(context, rightIntervalSplitRef);
+        connectOperators(rightMaterialize0Ref, rightIntervalSplitRef, context);
+        updateOperatorContext(context, rightMaterialize0Ref);
+        connectOperators(rightMaterialize1Ref, rightIntervalSplitRef, context);
+        updateOperatorContext(context, rightMaterialize1Ref);
+        connectOperators(rightMaterialize2Ref, rightIntervalSplitRef, context);
+        updateOperatorContext(context, rightMaterialize2Ref);
+
+        connectOperators(rightStartsSplitRef, rightMaterialize0Ref, context);
+        updateOperatorContext(context, rightStartsSplitRef);
+
+        // Connect left and right starts path
+        connectOperators(startsJoinRef, leftStartsSplitRef, context);
+        connectOperators(startsJoinRef, rightStartsSplitRef, context);
+        updateOperatorContext(context, startsJoinRef);
+
+        // Connect left ends path
+        connectOperators(leftEndsJoinRef, leftMaterialize1Ref, context);
+        connectOperators(leftEndsJoinRef, rightStartsSplitRef, context);
+        updateOperatorContext(context, leftEndsJoinRef);
+        connectOperators(union1Ref, startsJoinRef, context);
+        connectOperators(union1Ref, leftEndsJoinRef, context);
+        updateOperatorContext(context, union1Ref);
+
+        // Connect left covers path
+        connectOperators(leftCoversJoinRef, leftMaterialize2Ref, context);
+        connectOperators(leftCoversJoinRef, rightStartsSplitRef, context);
+        updateOperatorContext(context, leftCoversJoinRef);
+        connectOperators(union2Ref, union1Ref, context);
+        connectOperators(union2Ref, leftCoversJoinRef, context);
+        updateOperatorContext(context, union2Ref);
+
+        // Connect right ends path
+        connectOperators(rightEndsJoinRef, leftStartsSplitRef, context);
+        connectOperators(rightEndsJoinRef, rightMaterialize1Ref, context);
+        updateOperatorContext(context, rightEndsJoinRef);
+        connectOperators(union3Ref, union2Ref, context);
+        connectOperators(union3Ref, rightEndsJoinRef, context);
+        updateOperatorContext(context, union3Ref);
+
+        // Connect right covers path
+        connectOperators(rightCoversJoinRef, leftStartsSplitRef, context);
+        connectOperators(rightCoversJoinRef, rightMaterialize2Ref, context);
+        updateOperatorContext(context, rightCoversJoinRef);
+        connectOperators(union4Ref, union3Ref, context);
+        connectOperators(union4Ref, rightCoversJoinRef, context);
+        updateOperatorContext(context, union4Ref);
+
+        // Update context
+        opRef.setValue(union4Ref.getValue());
+
+        context.addToDontApplySet(this, startsJoin);
+        context.addToDontApplySet(this, leftCoversJoin);
+        context.addToDontApplySet(this, rightCoversJoin);
+        context.addToDontApplySet(this, leftCoversJoin);
+        context.addToDontApplySet(this, rightEndsJoin);
+
+        context.addToDontApplySet(this, union1);
+        context.addToDontApplySet(this, union2);
+        context.addToDontApplySet(this, union3);
+        context.addToDontApplySet(this, union4);
+        return true;
+    }
+
+    private void updateOperatorContext(IOptimizationContext context, Mutable<ILogicalOperator> operatorRef)
+            throws AlgebricksException {
+//        operatorRef.getValue().recomputeSchema();
+//        operatorRef.getValue().computeDeliveredPhysicalProperties(context);
+        context.computeAndSetTypeEnvironmentForOperator(operatorRef.getValue());
+        OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) operatorRef, context);
+    }
+
+    private LogicalVariable getSortKey(ILogicalOperator op) {
+        if (op.getOperatorTag() != LogicalOperatorTag.ORDER) {
+            return null;
+        }
+        OrderOperator oo = (OrderOperator) op;
+        List<Pair<IOrder, Mutable<ILogicalExpression>>> order = oo.getOrderExpressions();
+        Mutable<ILogicalExpression> sortLe = order.get(0).second;
+        if (sortLe.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            return ((VariableReferenceExpression) sortLe.getValue()).getVariableReference();
+        }
+        return null;
+    }
+
+    private LogicalVariable getJoinKey(ILogicalExpression expr, int branch) {
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return null;
+        }
+        // Check whether the function is a function we want to alter.
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        if (!INTERVAL_JOIN_CONDITIONS.contains(funcExpr.getFunctionIdentifier())) {
+            return null;
+        }
+        ILogicalExpression funcArg = funcExpr.getArguments().get(branch).getValue();
+        if (funcArg instanceof VariableReferenceExpression) {
+            return ((VariableReferenceExpression) funcArg).getVariableReference();
+        }
+        return null;
+    }
+
+    private void connectOperators(Mutable<ILogicalOperator> child, Mutable<ILogicalOperator> parent,
+            IOptimizationContext context) throws AlgebricksException {
+        if (parent.getValue().getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+            ILogicalOperator eo = getExchangeOperator(child.getValue().getExecutionMode());
+            Mutable<ILogicalOperator> eoRef = new MutableObject<>(eo);
+            eo.getInputs().add(parent);
+            if (parent.getValue().getOperatorTag() == LogicalOperatorTag.REPLICATE) {
+                ReplicateOperator ro = (ReplicateOperator) parent.getValue();
+                ro.getOutputs().add(eoRef);
+            }
+            child.getValue().getInputs().add(eoRef);
+            context.computeAndSetTypeEnvironmentForOperator(eo);
+            context.computeAndSetTypeEnvironmentForOperator(child.getValue());
+            OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull((AbstractLogicalOperator) eo, context);
+        } else {
+            if (parent.getValue().getOperatorTag() == LogicalOperatorTag.REPLICATE) {
+                ReplicateOperator ro = (ReplicateOperator) parent.getValue();
+                ro.getOutputs().add(child);
+            }
+            child.getValue().getInputs().add(parent);
+            context.computeAndSetTypeEnvironmentForOperator(child.getValue());
+        }
+    }
+
+    private ILogicalOperator getExchangeOperator(ExecutionMode mode) {
+        ExchangeOperator eo = new ExchangeOperator();
+        eo.setPhysicalOperator(new OneToOneExchangePOperator());
+        eo.setExecutionMode(mode);
+        return eo;
+    }
+
+    private ReplicateOperator getIntervalSplitOperator(LogicalVariable key, RangeId rangeId, ExecutionMode mode) {
+        List<LogicalVariable> joinKeyLogicalVars = new ArrayList<>();
+        joinKeyLogicalVars.add(key);
+        //create the logical and physical operator
+        boolean[] flags = new boolean[2];
+        for (int i = 0; i < flags.length; ++i) {
+            flags[i] = true;
+        }
+        ReplicateOperator splitOperator = new ReplicateOperator(flags.length, flags);
+        IntervalLocalRangeSplitterPOperator splitPOperator = new IntervalLocalRangeSplitterPOperator(joinKeyLogicalVars,
+                rangeId);
+        splitOperator.setPhysicalOperator(splitPOperator);
+        splitOperator.setExecutionMode(mode);
+        return splitOperator;
+    }
+
+    private ReplicateOperator getReplicateOperator(int outputArity, ExecutionMode mode) {
+        boolean[] flags = new boolean[outputArity];
+        for (int i = 0; i < flags.length; ++i) {
+            flags[i] = true;
+        }
+        ReplicateOperator ro = new ReplicateOperator(flags.length, flags);
+        ReplicatePOperator rpo = new ReplicatePOperator();
+        ro.setPhysicalOperator(rpo);
+        ro.setExecutionMode(mode);
+        return ro;
+    }
+
+    private ILogicalOperator getMaterializeOperator(ExecutionMode mode) {
+        MaterializeOperator mo = new MaterializeOperator();
+        MaterializePOperator mpo = new MaterializePOperator(false);
+        mo.setPhysicalOperator(mpo);
+        mo.setExecutionMode(mode);
+        return mo;
+    }
+
+    private ILogicalOperator getNestedLoop(Mutable<ILogicalExpression> condition, IOptimizationContext context,
+            ExecutionMode mode) {
+        int memoryJoinSize = context.getPhysicalOptimizationConfig().getMaxFramesForJoin();
+        InnerJoinOperator ijo = new InnerJoinOperator(condition);
+        NestedLoopJoinPOperator nljpo = new NestedLoopJoinPOperator(JoinKind.INNER, JoinPartitioningType.BROADCAST,
+                memoryJoinSize);
+        ijo.setPhysicalOperator(nljpo);
+        ijo.setExecutionMode(mode);
+        return ijo;
+    }
+
+    private ILogicalOperator getIntervalJoin(ILogicalOperator op, IOptimizationContext context, ExecutionMode mode) {
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return null;
+        }
+        InnerJoinOperator ijo = (InnerJoinOperator) op;
+        InnerJoinOperator ijoClone = new InnerJoinOperator(ijo.getCondition());
+
+        int memoryJoinSize = context.getPhysicalOptimizationConfig().getMaxFramesForJoin();
+        IPhysicalOperator joinPo = ijo.getPhysicalOperator();
+        if (joinPo.getOperatorTag() == PhysicalOperatorTag.MERGE_JOIN) {
+            MergeJoinPOperator mjpo = (MergeJoinPOperator) joinPo;
+            MergeJoinPOperator mjpoClone = new MergeJoinPOperator(mjpo.getKind(), mjpo.getPartitioningType(),
+                    mjpo.getKeysLeftBranch(), mjpo.getKeysRightBranch(), memoryJoinSize,
+                    mjpo.getMergeJoinCheckerFactory(), mjpo.getLeftRangeId(), mjpo.getRightRangeId(),
+                    mjpo.getRangeMapHint());
+            ijoClone.setPhysicalOperator(mjpoClone);
+        } else if (joinPo.getOperatorTag() == PhysicalOperatorTag.EXTENSION_OPERATOR) {
+            if (joinPo instanceof IntervalIndexJoinPOperator) {
+                IntervalIndexJoinPOperator iijpo = (IntervalIndexJoinPOperator) joinPo;
+                IntervalIndexJoinPOperator iijpoClone = new IntervalIndexJoinPOperator(iijpo.getKind(),
+                        iijpo.getPartitioningType(), iijpo.getKeysLeftBranch(), iijpo.getKeysRightBranch(),
+                        memoryJoinSize, iijpo.getIntervalMergeJoinCheckerFactory(), iijpo.getLeftRangeId(),
+                        iijpo.getRightRangeId(), iijpo.getRangeMapHint());
+                ijoClone.setPhysicalOperator(iijpoClone);
+            } else if (joinPo instanceof IntervalPartitionJoinPOperator) {
+                IntervalPartitionJoinPOperator ipjpo = (IntervalPartitionJoinPOperator) joinPo;
+                IntervalPartitionJoinPOperator iijpoClone = new IntervalPartitionJoinPOperator(ipjpo.getKind(),
+                        ipjpo.getPartitioningType(), ipjpo.getKeysLeftBranch(), ipjpo.getKeysRightBranch(),
+                        memoryJoinSize, ipjpo.getK(), ipjpo.getIntervalMergeJoinCheckerFactory(),
+                        ipjpo.getLeftPartitionVar(), ipjpo.getRightPartitionVar(), ipjpo.getLeftRangeId(),
+                        ipjpo.getRightRangeId(), ipjpo.getRangeMapHint());
+                ijoClone.setPhysicalOperator(iijpoClone);
+            } else {
+                return null;
+            }
+        } else {
+            return null;
+        }
+        ijoClone.setExecutionMode(mode);
+        return ijoClone;
+    }
+
+    private ILogicalOperator getUnionOperator(Set<LogicalVariable> localLiveVars, ExecutionMode mode) {
+        List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<>();
+        for (LogicalVariable lv : localLiveVars) {
+            varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(lv, lv, lv));
+        }
+        UnionAllOperator uao = new UnionAllOperator(varMap);
+        uao.setPhysicalOperator(new UnionAllPOperator());
+        uao.setExecutionMode(mode);
+        return uao;
+    }
+
+    private boolean isIntervalJoin(ILogicalOperator op) {
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+        // TODO add check for condition.
+        InnerJoinOperator ijo = (InnerJoinOperator) op;
+        if (ijo.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.MERGE_JOIN) {
+            return true;
+        }
+        if (ijo.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.EXTENSION_OPERATOR) {
+            return true;
+        }
+        return false;
+    }
+
+    private RangeId getRangeMapForBranch(ILogicalOperator op) {
+        if (op.getOperatorTag() != LogicalOperatorTag.EXCHANGE) {
+            return null;
+        }
+        ExchangeOperator exchangeLeft = (ExchangeOperator) op;
+        if (exchangeLeft.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.RANGE_PARTITION_EXCHANGE) {
+            return null;
+        }
+        RangePartitionExchangePOperator exchangeLeftPO = (RangePartitionExchangePOperator) exchangeLeft
+                .getPhysicalOperator();
+        if (exchangeLeftPO.getRangeType() != RangePartitioningType.SPLIT) {
+            return null;
+        }
+        return exchangeLeftPO.getRangeId();
+    }
+
+}
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 677b9a7..6de1dae 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
@@ -33,6 +33,7 @@
 import org.apache.asterix.optimizer.rules.am.BTreeJobGenParams;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.asterix.optimizer.rules.util.JoinUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -62,7 +63,6 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
-import org.apache.hyracks.algebricks.rewriter.util.JoinUtils;
 
 public class SetAsterixPhysicalOperatorsRule implements IAlgebraicRewriteRule {
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 4a79387..72c47c3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -212,6 +213,11 @@
         }
 
         @Override
+        public Void visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+            return null;
+        }
+
+        @Override
         public Void visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
             return null;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 874cc7c..ead78b7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -69,6 +69,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -523,6 +524,11 @@
     }
 
     @Override
+    public ILogicalOperator visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
+    @Override
     public ILogicalOperator visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
         throw new UnsupportedOperationException("Script operators in a subplan are not supported!");
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index eeb2c2a..ccdb41d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -54,6 +54,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -291,6 +292,11 @@
     }
 
     @Override
+    public ILogicalOperator visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return visitSingleInputOperator(op);
+    }
+
+    @Override
     public ILogicalOperator visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
         throw new UnsupportedOperationException("Script operators in a subplan are not supported!");
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index ccf0aeb..75a42a5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -39,6 +39,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -168,6 +169,11 @@
     }
 
     @Override
+    public Boolean visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return visitInputs(op);
+    }
+
+    @Override
     public Boolean visitScriptOperator(ScriptOperator op, Void arg) throws AlgebricksException {
         return false;
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
index 15cda86..b6e821f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
@@ -23,6 +23,7 @@
 import java.util.List;
 import java.util.Set;
 
+import org.apache.asterix.common.annotations.IntervalJoinExpressionAnnotation;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -34,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.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.functions.AlgebricksBuiltinFunctions;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -89,6 +91,15 @@
     }
 
     private boolean hasTranslatableInterval(AbstractFunctionCallExpression funcExpr) {
+        for (Object key : funcExpr.getAnnotations().keySet()) {
+            IExpressionAnnotation annot = funcExpr.getAnnotations().get(key);
+            if (annot instanceof IntervalJoinExpressionAnnotation) {
+                IntervalJoinExpressionAnnotation ijea = (IntervalJoinExpressionAnnotation) annot;
+                if (ijea.isRawJoin()) {
+                    return true;
+                }
+            }
+        }
         if (TRANSLATABLE_INTERVALS.contains(funcExpr.getFunctionIdentifier())) {
             return true;
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java
new file mode 100644
index 0000000..fb70aea
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java
@@ -0,0 +1,370 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.algebra.operators.physical.IntervalIndexJoinPOperator;
+import org.apache.asterix.algebra.operators.physical.IntervalPartitionJoinPOperator;
+import org.apache.asterix.common.annotations.IntervalJoinExpressionAnnotation;
+import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.runtime.operators.joins.AfterIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.BeforeIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.CoveredByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.CoversIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.EndedByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.EndsIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.MeetsIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.MetByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.OverlappedByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.OverlappingIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.OverlapsIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.StartedByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.StartsIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.intervalpartition.IntervalPartitionUtil;
+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.ConstantExpression;
+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.RangeForwardOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.MergeJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangeForwardPOperator;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+
+public class JoinUtils {
+
+    private static final int LEFT = 0;
+    private static final int RIGHT = 1;
+
+    private static final Logger LOGGER = Logger.getLogger(JoinUtils.class.getName());
+
+    private static final Map<FunctionIdentifier, FunctionIdentifier> INTERVAL_JOIN_CONDITIONS = new HashMap<>();
+    static {
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_AFTER, AsterixBuiltinFunctions.INTERVAL_BEFORE);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_BEFORE, AsterixBuiltinFunctions.INTERVAL_AFTER);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_COVERED_BY,
+                AsterixBuiltinFunctions.INTERVAL_COVERS);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_COVERS,
+                AsterixBuiltinFunctions.INTERVAL_COVERED_BY);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_ENDED_BY, AsterixBuiltinFunctions.INTERVAL_ENDS);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_ENDS, AsterixBuiltinFunctions.INTERVAL_ENDED_BY);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_MEETS, AsterixBuiltinFunctions.INTERVAL_MET_BY);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_MET_BY, AsterixBuiltinFunctions.INTERVAL_MEETS);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY,
+                AsterixBuiltinFunctions.INTERVAL_OVERLAPS);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_OVERLAPPING,
+                AsterixBuiltinFunctions.INTERVAL_OVERLAPPING);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_OVERLAPS,
+                AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_STARTED_BY,
+                AsterixBuiltinFunctions.INTERVAL_STARTS);
+        INTERVAL_JOIN_CONDITIONS.put(AsterixBuiltinFunctions.INTERVAL_STARTS,
+                AsterixBuiltinFunctions.INTERVAL_STARTED_BY);
+    }
+
+    private JoinUtils() {
+    }
+
+    public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalExpression conditionLE = op.getCondition().getValue();
+        if (conditionLE.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return;
+        }
+        List<LogicalVariable> sideLeft = new LinkedList<>();
+        List<LogicalVariable> sideRight = new LinkedList<>();
+        List<LogicalVariable> varsLeft = op.getInputs().get(LEFT).getValue().getSchema();
+        List<LogicalVariable> varsRight = op.getInputs().get(RIGHT).getValue().getSchema();
+        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) conditionLE;
+        FunctionIdentifier fi = isIntervalJoinCondition(fexp, varsLeft, varsRight, sideLeft, sideRight);
+        if (fi != null) {
+            IntervalJoinExpressionAnnotation ijea = getIntervalJoinAnnotation(fexp);
+            if (ijea == null) {
+                // Use default join method.
+                return;
+            }
+            if (ijea.isMergeJoin()) {
+                // Sort Merge.
+                LOGGER.fine("Interval Join - Merge");
+                setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, ijea, context);
+            } else if (ijea.isPartitionJoin()) {
+                // Overlapping Interval Partition.
+                LOGGER.fine("Interval Join - Cluster Parititioning");
+                setIntervalPartitionJoinOp(op, fi, sideLeft, sideRight, ijea, context);
+            } else if (ijea.isSpatialJoin()) {
+                // Spatial Partition.
+                LOGGER.fine("Interval Join - Spatial Partitioning");
+            } else if (ijea.isIndexJoin()) {
+                // Endpoint Index.
+                LOGGER.fine("Interval Join - Endpoint Index");
+                setIntervalIndexJoinOp(op, fi, sideLeft, sideRight, ijea, context);
+            }
+        }
+    }
+
+    private static IntervalJoinExpressionAnnotation getIntervalJoinAnnotation(AbstractFunctionCallExpression fexp) {
+        Iterator<IExpressionAnnotation> annotationIter = fexp.getAnnotations().values().iterator();
+        while (annotationIter.hasNext()) {
+            IExpressionAnnotation annotation = annotationIter.next();
+            if (annotation instanceof IntervalJoinExpressionAnnotation) {
+                return (IntervalJoinExpressionAnnotation) annotation;
+            }
+        }
+        return null;
+    }
+
+    private static void setSortMergeIntervalJoinOp(AbstractBinaryJoinOperator op, FunctionIdentifier fi,
+            List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IntervalJoinExpressionAnnotation ijea,
+            IOptimizationContext context) throws AlgebricksException {
+        RangeId leftRangeId = context.newRangeId();
+        RangeId rightRangeId = context.newRangeId();
+        insertRangeForward(op, LEFT, leftRangeId, ijea.getRangeMap(), context);
+        insertRangeForward(op, RIGHT, rightRangeId, ijea.getRangeMap(), context);
+
+        IMergeJoinCheckerFactory mjcf = getIntervalMergeJoinCheckerFactory(fi, leftRangeId);
+        op.setPhysicalOperator(new MergeJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST, sideLeft,
+                sideRight, context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), mjcf, leftRangeId,
+                rightRangeId, ijea.getRangeMap()));
+    }
+
+    private static void setIntervalPartitionJoinOp(AbstractBinaryJoinOperator op, FunctionIdentifier fi,
+            List<LogicalVariable> leftKeys, List<LogicalVariable> rightKeys, IntervalJoinExpressionAnnotation ijea,
+            IOptimizationContext context) throws AlgebricksException {
+        long leftCount = ijea.getLeftRecordCount() > 0 ? ijea.getLeftRecordCount() : getCardinality(leftKeys, context);
+        long rightCount = ijea.getRightRecordCount() > 0 ? ijea.getRightRecordCount()
+                : getCardinality(rightKeys, context);
+        long leftMaxDuration = ijea.getLeftMaxDuration() > 0 ? ijea.getLeftMaxDuration()
+                : getMaxDuration(leftKeys, context);
+        long rightMaxDuration = ijea.getRightMaxDuration() > 0 ? ijea.getRightMaxDuration()
+                : getMaxDuration(rightKeys, context);
+        int tuplesPerFrame = ijea.getTuplesPerFrame() > 0 ? ijea.getTuplesPerFrame()
+                : context.getPhysicalOptimizationConfig().getMaxRecordsPerFrame();
+
+        int k = IntervalPartitionUtil.determineK(leftCount, leftMaxDuration, rightCount, rightMaxDuration,
+                tuplesPerFrame);
+        if (k <= 2) {
+            k = 3;
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("IntervalPartitionJoin has overridden the suggested value of k (" + k + ") with 3.");
+            }
+        }
+
+        RangeId leftRangeId = context.newRangeId();
+        RangeId rightRangeId = context.newRangeId();
+        insertRangeForward(op, LEFT, leftRangeId, ijea.getRangeMap(), context);
+        insertRangeForward(op, RIGHT, rightRangeId, ijea.getRangeMap(), context);
+
+        List<LogicalVariable> leftPartitionVar = Arrays.asList(context.newVar(), context.newVar());
+        List<LogicalVariable> rightPartitionVar = Arrays.asList(context.newVar(), context.newVar());
+        insertPartitionSortKey(op, LEFT, leftPartitionVar, leftKeys.get(0), leftRangeId, k, context);
+        insertPartitionSortKey(op, RIGHT, rightPartitionVar, rightKeys.get(0), rightRangeId, k, context);
+
+        IIntervalMergeJoinCheckerFactory mjcf = getIntervalMergeJoinCheckerFactory(fi, leftRangeId);
+        op.setPhysicalOperator(new IntervalPartitionJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST,
+                leftKeys, rightKeys, context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), k, mjcf,
+                leftPartitionVar, rightPartitionVar, leftRangeId, rightRangeId, ijea.getRangeMap()));
+    }
+
+    private static void insertRangeForward(AbstractBinaryJoinOperator op, int branch, RangeId rangeId,
+            IRangeMap rangeMap, IOptimizationContext context) throws AlgebricksException {
+        RangeForwardOperator rfo = new RangeForwardOperator(rangeId, rangeMap);
+        rfo.setExecutionMode(op.getExecutionMode());
+        rfo.getInputs().add(op.getInputs().get(branch));
+        RangeForwardPOperator rfpo = new RangeForwardPOperator(rangeId, rangeMap);
+        rfo.setPhysicalOperator(rfpo);
+        Mutable<ILogicalOperator> rfoRef = new MutableObject<>(rfo);
+        op.getInputs().set(branch, rfoRef);
+
+        context.computeAndSetTypeEnvironmentForOperator(rfo);
+    }
+
+    private static void insertPartitionSortKey(AbstractBinaryJoinOperator op, int branch,
+            List<LogicalVariable> partitionVars, LogicalVariable intervalVar, RangeId rangeId, int k,
+            IOptimizationContext context) throws AlgebricksException {
+        MutableObject<ILogicalExpression> intervalExp = new MutableObject<>(
+                new VariableReferenceExpression(intervalVar));
+        MutableObject<ILogicalExpression> rangeIdConstant = new MutableObject<>(
+                new ConstantExpression(new AsterixConstantValue(new AInt32(rangeId.getId()))));
+        MutableObject<ILogicalExpression> kConstant = new MutableObject<>(
+                new ConstantExpression(new AsterixConstantValue(new AInt32(k))));
+
+        List<Mutable<ILogicalExpression>> assignExps = new ArrayList<>();
+        // Start partition
+        IFunctionInfo startFi = FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.INTERVAL_PARTITION_JOIN_START);
+        @SuppressWarnings("unchecked")
+        ScalarFunctionCallExpression startPartitionExp = new ScalarFunctionCallExpression(startFi, intervalExp,
+                rangeIdConstant, kConstant);
+        assignExps.add(new MutableObject<ILogicalExpression>(startPartitionExp));
+        // End partition
+        IFunctionInfo endFi = FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.INTERVAL_PARTITION_JOIN_END);
+        @SuppressWarnings("unchecked")
+        ScalarFunctionCallExpression endPartitionExp = new ScalarFunctionCallExpression(endFi, intervalExp,
+                rangeIdConstant, kConstant);
+        assignExps.add(new MutableObject<ILogicalExpression>(endPartitionExp));
+
+        AssignOperator ao = new AssignOperator(partitionVars, assignExps);
+        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);
+    }
+
+    private static void setIntervalIndexJoinOp(AbstractBinaryJoinOperator op, FunctionIdentifier fi,
+            List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IntervalJoinExpressionAnnotation ijea,
+            IOptimizationContext context) throws AlgebricksException {
+        RangeId leftRangeId = context.newRangeId();
+        RangeId rightRangeId = context.newRangeId();
+        insertRangeForward(op, LEFT, leftRangeId, ijea.getRangeMap(), context);
+        insertRangeForward(op, RIGHT, rightRangeId, ijea.getRangeMap(), context);
+
+        IIntervalMergeJoinCheckerFactory mjcf = getIntervalMergeJoinCheckerFactory(fi, leftRangeId);
+        op.setPhysicalOperator(new IntervalIndexJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST,
+                sideLeft, sideRight, context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), mjcf, leftRangeId,
+                rightRangeId, ijea.getRangeMap()));
+    }
+
+    private static int getMaxDuration(List<LogicalVariable> lv, IOptimizationContext context) {
+        // TODO Base on real statistics
+        return context.getPhysicalOptimizationConfig().getMaxIntervalDuration();
+    }
+
+    private static int getCardinality(List<LogicalVariable> lv, IOptimizationContext context) {
+        // TODO Base on real statistics
+        return context.getPhysicalOptimizationConfig().getMaxFramesForJoinLeftInput();
+    }
+
+    private static FunctionIdentifier isIntervalJoinCondition(ILogicalExpression e,
+            Collection<LogicalVariable> inLeftAll, Collection<LogicalVariable> inRightAll,
+            Collection<LogicalVariable> outLeftFields, Collection<LogicalVariable> outRightFields) {
+        FunctionIdentifier fiReturn;
+        boolean switchArguments = false;
+        if (e.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            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 = reverseIntervalExpression(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;
+        } else {
+            return null;
+        }
+    }
+
+    private static IIntervalMergeJoinCheckerFactory getIntervalMergeJoinCheckerFactory(FunctionIdentifier fi,
+            RangeId rangeId) {
+        IIntervalMergeJoinCheckerFactory mjcf = new OverlappingIntervalMergeJoinCheckerFactory(rangeId);
+        if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+            mjcf = new OverlappedByIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPS)) {
+            mjcf = new OverlapsIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_COVERS)) {
+            mjcf = new CoversIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_COVERED_BY)) {
+            mjcf = new CoveredByIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_STARTS)) {
+            mjcf = new StartsIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_STARTED_BY)) {
+            mjcf = new StartedByIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_ENDS)) {
+            mjcf = new EndsIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_ENDED_BY)) {
+            mjcf = new EndedByIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_MEETS)) {
+            mjcf = new MeetsIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_MET_BY)) {
+            mjcf = new MetByIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_BEFORE)) {
+            mjcf = new BeforeIntervalMergeJoinCheckerFactory();
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_AFTER)) {
+            mjcf = new AfterIntervalMergeJoinCheckerFactory();
+        }
+        return mjcf;
+    }
+
+    private static boolean isIntervalFunction(FunctionIdentifier fi) {
+        return INTERVAL_JOIN_CONDITIONS.containsKey(fi);
+    }
+
+    private static FunctionIdentifier reverseIntervalExpression(FunctionIdentifier fi) {
+        if (INTERVAL_JOIN_CONDITIONS.containsKey(fi)) {
+            return INTERVAL_JOIN_CONDITIONS.get(fi);
+        }
+        return null;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
index dd45c86..9c93e07 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/util/FunctionCollection.java
@@ -292,13 +292,15 @@
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMeetsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalMetByDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlappedByDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlappingDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionJoinEndDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionJoinStartDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapBinsDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.ParseDateTimeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.temporal.ParseTimeDescriptor;
@@ -435,6 +437,10 @@
         temp.add(SimilarityJaccardPrefixDescriptor.FACTORY);
         temp.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
 
+        // Partition functions for interval partition join pre-sorting
+        temp.add(IntervalPartitionJoinStartDescriptor.FACTORY);
+        temp.add(IntervalPartitionJoinEndDescriptor.FACTORY);
+
         // functions that need generated class for null-handling.
         List<IFunctionDescriptorFactory> functionsToInjectUnkownHandling = new ArrayList<>();
 
@@ -641,7 +647,7 @@
         functionsToInjectUnkownHandling.add(IntervalMetByDescriptor.FACTORY);
         functionsToInjectUnkownHandling.add(IntervalOverlapsDescriptor.FACTORY);
         functionsToInjectUnkownHandling.add(IntervalOverlappedByDescriptor.FACTORY);
-        functionsToInjectUnkownHandling.add(OverlapDescriptor.FACTORY);
+        functionsToInjectUnkownHandling.add(IntervalOverlappingDescriptor.FACTORY);
         functionsToInjectUnkownHandling.add(IntervalStartsDescriptor.FACTORY);
         functionsToInjectUnkownHandling.add(IntervalStartedByDescriptor.FACTORY);
         functionsToInjectUnkownHandling.add(IntervalCoversDescriptor.FACTORY);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index eb1764c..a48a6aa 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -250,10 +250,15 @@
         int sortFrameLimit = (int) (compilerProperties.getSortMemorySize() / frameSize);
         int groupFrameLimit = (int) (compilerProperties.getGroupMemorySize() / frameSize);
         int joinFrameLimit = (int) (compilerProperties.getJoinMemorySize() / frameSize);
+        int joinFrameLeftInputLimit = (int) (compilerProperties.getJoinLeftInputSize() / frameSize);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setFrameSize(frameSize);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalSort(sortFrameLimit);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalGroupBy(groupFrameLimit);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesForJoin(joinFrameLimit);
+        OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesForJoinLeftInput(joinFrameLeftInputLimit);
+
+        int intervalMaxDuration = (int) (compilerProperties.getIntervalMaxDuration());
+        OptimizationConfUtil.getPhysicalOptimizationConfig().getMaxIntervalDuration(intervalMaxDuration);
 
         HeuristicCompilerFactoryBuilder builder =
                 new HeuristicCompilerFactoryBuilder(AqlOptimizationContextFactory.INSTANCE);
diff --git a/asterixdb/asterix-app/src/test/resources/logging.properties b/asterixdb/asterix-app/src/test/resources/logging.properties
index 5b5e3b6..baf225c 100644
--- a/asterixdb/asterix-app/src/test/resources/logging.properties
+++ b/asterixdb/asterix-app/src/test/resources/logging.properties
@@ -19,7 +19,7 @@
 #      Default Logging Configuration File
 #
 # You can use a different file by specifying a filename
-# with the java.util.logging.config.file system property.  
+# with the java.util.logging.config.file system property.
 # For example java -Djava.util.logging.config.file=myfile
 ############################################################
 
@@ -27,7 +27,7 @@
 #      Global properties
 ############################################################
 
-# "handlers" specifies a comma separated list of log Handler 
+# "handlers" specifies a comma separated list of log Handler
 # classes.  These handlers will be installed during VM startup.
 # Note that these classes must be on the system classpath.
 # By default we only configure a ConsoleHandler, which will only
@@ -79,8 +79,10 @@
 
 
 org.apache.asterix.test.level = INFO
-#org.apache.asterix.level = FINE
 #org.apache.hyracks.algebricks.level = FINE
 #org.apache.hyracks.level = INFO
+org.apache.hyracks.dataflow.level = FINE
 org.apache.asterix.test = INFO
 org.apache.asterix.installer.test = INFO
+org.apache.hyracks.dataflow.level = FINE
+org.apache.asterix.runtime.operators.joins.level = FINE
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric-desc.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric-desc.plan
index cdedfde..2ac6e96 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric-desc.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric-desc.plan
@@ -1,12 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- RANGE_PARTITION_MERGE_EXCHANGE [$$4(DESC)] SPLIT COUNT:3  |PARTITIONED|
-        -- STABLE_SORT [$$4(DESC)]  |PARTITIONED|
+      -- RANGE_PARTITION_MERGE_EXCHANGE [$$4(DESC)] PROJECT RangeId(0)  |PARTITIONED|
+        -- RANGE_FORWARD RangeId(0)  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN  |PARTITIONED|
+            -- STABLE_SORT [$$4(DESC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                      -- 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/range-connector/sort-hint-on-closed-numeric.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric.plan
index ba0dc6f..4b8bf86 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-numeric.plan
@@ -1,12 +1,14 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- RANGE_PARTITION_MERGE_EXCHANGE [$$4(ASC)] SPLIT COUNT:3  |PARTITIONED|
-        -- STABLE_SORT [$$4(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_MERGE_EXCHANGE [$$4(ASC)] PROJECT RangeId(0)  |PARTITIONED|
+        -- RANGE_FORWARD RangeId(0)  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- ASSIGN  |PARTITIONED|
-              -- STREAM_PROJECT  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- DATASOURCE_SCAN  |PARTITIONED|
+            -- STABLE_SORT [$$4(ASC)]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                      -- 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/range-connector/sort-hint-on-closed-string-desc.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string-desc.plan
index 3faa5ec..7732fbe 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string-desc.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string-desc.plan
@@ -1,9 +1,11 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- RANGE_PARTITION_MERGE_EXCHANGE [$$3(DESC)] SPLIT COUNT:3  |PARTITIONED|
-        -- STABLE_SORT [$$3(DESC)]  |PARTITIONED|
+      -- RANGE_PARTITION_MERGE_EXCHANGE [$$3(DESC)] PROJECT RangeId(0)  |PARTITIONED|
+        -- RANGE_FORWARD RangeId(0)  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- DATASOURCE_SCAN  |PARTITIONED|
+            -- STABLE_SORT [$$3(DESC)]  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                -- 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/range-connector/sort-hint-on-closed-string.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string.plan
index e0cffaa..fe51372 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/range-connector/sort-hint-on-closed-string.plan
@@ -1,7 +1,9 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- RANGE_PARTITION_MERGE_EXCHANGE [$$3(ASC)] SPLIT COUNT:3  |PARTITIONED|
-        -- DATASOURCE_SCAN  |PARTITIONED|
+      -- RANGE_PARTITION_MERGE_EXCHANGE [$$3(ASC)] PROJECT RangeId(0)  |PARTITIONED|
+        -- RANGE_FORWARD RangeId(0)  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+            -- DATASOURCE_SCAN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_01/order-by-exception_01.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_01/order-by-exception_01.4.query.aql
index 8667a4c..3f89553 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_01/order-by-exception_01.4.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_01/order-by-exception_01.4.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range ["Ci", "Nb", "F"] */
+/*+ range ["Aa", "Ci", "Nb", "F", "Zb"] */
 order by $user.screen-name
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_02/order-by-exception_02.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_02/order-by-exception_02.4.query.aql
index 02f55c6..9520dd0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_02/order-by-exception_02.4.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by-exception_02/order-by-exception_02.4.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range ["Ci", "Nb", "F"] */
+/*+ range ["Aa", "Ci", "Nb", "F", "Zb"] */
 order by $user.screen-name desc
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.4.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.4.query.aql
index ac6f643..1fffdc2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.4.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.4.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range ["Ci", "F", "Nb"] */
+/*+ range ["Aa", "Ci", "F", "Nb", "Zz"] */
 order by $user.screen-name
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.5.query.aql
index 10f1ba1..e6dcfbe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.5.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.5.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range ["Nb", "F", "Ci"] */
+/*+ range ["Zz", "Nb", "F", "Ci", "Aa"] */
 order by $user.screen-name desc
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.6.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.6.query.aql
index 628a118..88bad1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.6.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.6.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range [100, 150, 400] */
+/*+ range [0, 100, 150, 400, 500] */
 order by $user.friends_count
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.7.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.7.query.aql
index d478451..1fa00ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.7.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/range-hints/order-by/order-by.7.query.aql
@@ -19,6 +19,6 @@
 use dataverse TinySocial;
 
 for $user in dataset TwitterUsers
-/*+ range [400, 150, 100] */
+/*+ range [500, 400, 150, 100, 0] */
 order by $user.friends_count desc
 return $user;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.query.aql
similarity index 86%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.3.query.aql
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.query.aql
index 62ed6a9..8609fec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.query.aql
@@ -16,8 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-use dataverse test;
-
 let $itv1 := interval(date("2010-10-30"), date("2010-12-21"))
 let $itv2 := interval(date("2011-10-30"), date("2012-10-21"))
 let $itv3 := interval(date("2010-12-21"), date("2013-01-01"))
@@ -41,8 +39,8 @@
 let $blnOverlapped1 := interval-overlapped-by($itv8, $itv7)
 let $blnOverlaps2 := interval-overlaps($itv9, $itv8)
 let $blnOverlapped2 := interval-overlapped-by($itv8, $itv9)
-let $blnOverlap1 := interval-overlapping($itv9, $itv7)
-let $blnOverlap2 := interval-overlapping($itv9, $itv8)
+let $blnOverlapping1 := interval-overlapping($itv9, $itv7)
+let $blnOverlapping2 := interval-overlapping($itv9, $itv8)
 
 let $itv10 := interval(date("2010-10-30"), date("2010-11-30"))
 let $blnStarts1 := interval-starts($itv10, $itv1)
@@ -68,4 +66,4 @@
 let $null2 := interval-covered-by($itv11, null)
 let $null3 := interval-overlapping(null, null)
 
-return { "before1" : $blnBefore1, "before2" : $blnBefore2, "after1" : $blnAfter1, "after2" : $blnAfter2, "meet1" : $blnMeet1, "meet2" : $blnMeet2, "metby1" : $blnMetBy1, "metby2" : $blnMetBy2, "overlaps1" : $blnOverlaps1, "overlaps2" : $blnOverlaps2, "overlapped1" : $blnOverlapped1, "overlapped2" : $blnOverlapped2, "overlap1" : $blnOverlap1, "overlap2" : $blnOverlap2, "starts1" : $blnStarts1, "starts2" : $blnStarts2, "startedby1" : $blnStartedBy1, "startedby2" : $blnStartedBy2, "covers1" : $blnCovers1, "covers2" : $blnCovers2, "coveredby1" : $blnCoveredBy1, "coveredby2" : $blnCoveredBy2, "ends1" : $blnEnds1, "ends2" : $blnEnds2, "endedby1" : $blnEndedBy1, "endedby2" : $blnEndedBy2, "null1": $null1, "null2": $null2, "null3": $null3 }
+return { "before1" : $blnBefore1, "before2" : $blnBefore2, "after1" : $blnAfter1, "after2" : $blnAfter2, "meet1" : $blnMeet1, "meet2" : $blnMeet2, "metby1" : $blnMetBy1, "metby2" : $blnMetBy2, "overlaps1" : $blnOverlaps1, "overlaps2" : $blnOverlaps2, "overlapped1" : $blnOverlapped1, "overlapped2" : $blnOverlapped2, "overlapping1" : $blnOverlapping1, "overlapping2" : $blnOverlapping2, "starts1" : $blnStarts1, "starts2" : $blnStarts2, "startedby1" : $blnStartedBy1, "startedby2" : $blnStartedBy2, "covers1" : $blnCovers1, "covers2" : $blnCovers2, "coveredby1" : $blnCoveredBy1, "coveredby2" : $blnCoveredBy2, "ends1" : $blnEnds1, "ends2" : $blnEnds2, "endedby1" : $blnEndedBy1, "endedby2" : $blnEndedBy2, "null1": $null1, "null2": $null2, "null3": $null3 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.10.query.aql
index 500ac03..c72ed10 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-before($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.11.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.11.query.aql
index 500ac03..19aef04 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-after($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.12.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.12.query.aql
index 500ac03..ffc82ff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-after($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.13.query.aql
index 500ac03..b58f95d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-before($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.14.query.aql
index 500ac03..9ef6fe8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-before($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.15.query.aql
index 500ac03..2548f30 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-after($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.16.query.aql
index 500ac03..851d50c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-after($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.17.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.17.query.aql
index 500ac03..300b229 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-before($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.18.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.18.query.aql
index 500ac03..12055d3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-before($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.19.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.19.query.aql
index 500ac03..4bcf34e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-after($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.20.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.20.query.aql
index 500ac03..ff9b2dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-after($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.5.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.5.query.aql
index 500ac03..10f669d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-before($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.6.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.6.query.aql
index 500ac03..17eb00c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-before($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.7.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.7.query.aql
index 500ac03..65b2dcc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-after($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.8.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.8.query.aql
index 500ac03..a85ac7b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-before($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.9.query.aql
index 500ac03..0973c08 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_before/interval_before.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-before($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.10.query.aql
index 500ac03..1dabed8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-covers($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.11.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.11.query.aql
index 500ac03..2fb501e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-covered-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.12.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.12.query.aql
index 500ac03..18dd4a4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-covered-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.13.query.aql
index 500ac03..84f5223 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.14.query.aql
index 500ac03..e2342fa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.15.query.aql
index 500ac03..83733f3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-covered-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.16.query.aql
index 500ac03..41b1044 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-covered-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.17.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.17.query.aql
index 500ac03..4762a26 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.18.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.18.query.aql
index 500ac03..8a4d1da 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.19.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.19.query.aql
index 500ac03..27d3f40 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-covered-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.20.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.20.query.aql
index 500ac03..063b138 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-covered-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.5.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.5.query.aql
index 500ac03..a737d15 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.6.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.6.query.aql
index 500ac03..f902345 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.7.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.7.query.aql
index 500ac03..d9117d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-covered-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.8.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.8.query.aql
index 500ac03..ef8cf01 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-covers($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.9.query.aql
index 500ac03..4111786 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_covers/interval_covers.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-covers($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.10.query.aql
index 500ac03..dcca42b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-ends($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.11.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.11.query.aql
index 500ac03..f5791a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-ended-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.12.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.12.query.aql
index 500ac03..094d777 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-ended-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.13.query.aql
index 500ac03..8333301 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.14.query.aql
index 500ac03..f75f95e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.15.query.aql
index 500ac03..5250fa5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-ended-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.16.query.aql
index 500ac03..e84535e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-ended-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.17.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.17.query.aql
index 500ac03..2d41384 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.18.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.18.query.aql
index 500ac03..884e104 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.19.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.19.query.aql
index 500ac03..5ede13c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-ended-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.20.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.20.query.aql
index 500ac03..2ff6335 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-ended-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.5.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.5.query.aql
index 500ac03..8a1a64a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.6.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.6.query.aql
index 500ac03..186170d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.7.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.7.query.aql
index 500ac03..71c4a05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-ended-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.8.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.8.query.aql
index 500ac03..1f62dc5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-ends($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.9.query.aql
index 500ac03..2e80169 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_ends/interval_ends.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-ends($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.10.query.aql
index 500ac03..5f7e1a0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-meets($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.11.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.11.query.aql
index 500ac03..b599df5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-met-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.12.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.12.query.aql
index 500ac03..51bacda 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-met-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.13.query.aql
index 500ac03..e754b1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.14.query.aql
index 500ac03..7c70dd7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.15.query.aql
index 500ac03..a22459c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-met-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.16.query.aql
index 500ac03..5ba5404 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-met-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.17.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.17.query.aql
index 500ac03..9945ffc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.18.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.18.query.aql
index 500ac03..52c1411 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.19.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.19.query.aql
index 500ac03..674358e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-met-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.20.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.20.query.aql
index 500ac03..b9dea15 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-met-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.5.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.5.query.aql
index 500ac03..ccaff15 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.6.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.6.query.aql
index 500ac03..19ecf6c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.7.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.7.query.aql
index 500ac03..26ef143 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-met-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.8.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.8.query.aql
index 500ac03..987a550 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-meets($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.9.query.aql
index 500ac03..0f01bf7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_meets/interval_meets.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-meets($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.aql
index 500ac03..cfb050b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlapping($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.aql
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.aql
index 500ac03..7cf839d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000,11000,12000,14000,15000] 7 7 400 400 300 */ interval-overlapping($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.aql
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.aql
index 500ac03..aa55cec 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000,11000,12000,14000,15000] 7 7 400 400 300 */ interval-overlapping($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.aql
index 500ac03..2132136 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapping($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.aql
index 500ac03..7e2b21a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapping($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.aql
index 500ac03..4e736d3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapping($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.aql
index 500ac03..38e0860 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapping($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.aql
index 500ac03..2525c8d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapping($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.aql
index 500ac03..e2f2892 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlapping/interval_overlapping.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlapping($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.aql
index 500ac03..f61cf66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlaps($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.aql
index 500ac03..dc2b7b0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlapped-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.aql
index 500ac03..401d1f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlapped-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.aql
index 500ac03..8591fae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.aql
index 500ac03..cf9bffc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.15.query.aql
index 500ac03..4b73b89 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapped-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.16.query.aql
index 500ac03..f52475b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapped-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.17.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.17.query.aql
index 500ac03..4e3e536 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.18.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.18.query.aql
index 500ac03..0fb5293 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.19.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.19.query.aql
index 500ac03..ade330e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapped-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.20.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.20.query.aql
index 500ac03..3d6b220 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapped-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
index 500ac03..7dc905b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
index 500ac03..9553681 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
index 500ac03..0bb14ea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlapped-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
index 500ac03..0d411ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-overlaps($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.aql
index 500ac03..b541193 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-overlaps($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.10.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.10.query.aql
index 500ac03..ff51f44 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.10.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-starts($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.11.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.11.query.aql
index 500ac03..6e07e04 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.11.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-started-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.12.query.aql
similarity index 68%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.12.query.aql
index 500ac03..eab99f4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.12.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-started-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.13.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.13.query.aql
index 500ac03..03ad672 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.13.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.14.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.14.query.aql
index 500ac03..07e4f9b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.14.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.15.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.15.query.aql
index 500ac03..4585b88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.15.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-started-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.16.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.16.query.aql
index 500ac03..4910efa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.16.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-partition-join [10000, 11000, 12000, 14000, 15000] */ interval-started-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.17.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.17.query.aql
index 500ac03..2841389 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.17.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.18.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.18.query.aql
index 500ac03..835d3ae 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.18.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.19.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.19.query.aql
index 500ac03..aa6f807 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.19.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-started-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.20.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.20.query.aql
index 500ac03..8f194ce 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.20.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-index-join [10000, 11000, 12000, 14000, 15000] */ interval-started-by($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.5.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.5.query.aql
index 500ac03..487da03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.5.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.6.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.6.query.aql
index 500ac03..f6e789f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.6.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.7.query.aql
similarity index 66%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.7.query.aql
index 500ac03..22d3d97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.7.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-started-by($f.employment, $d.attendance)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.8.query.aql
similarity index 67%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.8.query.aql
index 500ac03..39a6f19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.8.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [10000, 11000, 12000, 14000, 15000] */ interval-starts($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.9.query.aql
similarity index 69%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.9.query.aql
index 500ac03..fd0b9d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_starts/interval_starts.9.query.aql
@@ -17,10 +17,16 @@
  * under the License.
  */
 /*
- * Description      :   Check temporal functions for interval
+ * Description      :   Check temporal join functionality for interval
  * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
+ * Date             :   26th Jun, 2015
  */
 
-drop dataverse test if exists;
-create dataverse test;
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-raw-join */ interval-starts($d.attendance, $f.employment)
+/*+ range ["F", "L", "R"] */
+order by $f.name, $d.name
+return { "staff" : $f.name, "student" : $d.name }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.query.aql
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.3.query.aql
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.query.aql
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.2.update.aql
deleted file mode 100644
index f686b54..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.2.update.aql
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/**
- * overlap_bins test case: test the overlap_bins
- * Expected result: success
- **/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql
index db9fe58..061a0bf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tinysocial/tinysocial-suite/tinysocial-suite.11.query.aql
@@ -17,7 +17,7 @@
  * under the License.
  */
 /**
-* Query 5 - Fuzzy Join 
+* Query 5 - Fuzzy Join
 */
 
 use dataverse TinySocial;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_functions/interval_functions.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_functions/interval_functions.3.query.sqlpp
index 55ddb57..cf3ae8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_functions/interval_functions.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/interval_functions/interval_functions.3.query.sqlpp
@@ -42,8 +42,8 @@
 , blnOverlapped1 = `interval-overlapped-by`(itv8, itv7)
 , blnOverlaps2 = `interval-overlaps`(itv9, itv8)
 , blnOverlapped2 = `interval-overlapped-by`(itv8, itv9)
-, blnOverlap1 = `interval-overlapping`(itv9, itv7)
-, blnOverlap2 = `interval-overlapping`(itv9, itv8)
+, blnOverlapping1 = `interval-overlapping`(itv9, itv7)
+, blnOverlapping2 = `interval-overlapping`(itv9, itv8)
 
 , itv10 = interval(date('2010-10-30'), date('2010-11-30'))
 , blnStarts1 = `interval-starts`(itv10, itv1)
@@ -69,5 +69,5 @@
 , null2 = `interval-covered-by`(itv11, null)
 , null3 = `interval-overlapping`(null, null)
 
-select element { 'before1' : blnBefore1, 'before2' : blnBefore2, 'after1' : blnAfter1, 'after2' : blnAfter2, 'meet1' : blnMeet1, 'meet2' : blnMeet2, 'metby1' : blnMetBy1, 'metby2' : blnMetBy2, 'overlaps1' : blnOverlaps1, 'overlaps2' : blnOverlaps2, 'overlapped1' : blnOverlapped1, 'overlapped2' : blnOverlapped2, 'overlap1' : blnOverlap1, 'overlap2' : blnOverlap2, 'starts1' : blnStarts1, 'starts2' : blnStarts2, 'startedby1' : blnStartedBy1, 'startedby2' : blnStartedBy2, 'covers1' : blnCovers1, 'covers2' : blnCovers2, 'coveredby1' : blnCoveredBy1, 'coveredby2' : blnCoveredBy2, 'ends1' : blnEnds1, 'ends2' : blnEnds2, 'endedby1' : blnEndedBy1, 'endedby2' : blnEndedBy2, 'null1': null1, 'null2': null2, 'null3': null3 }
+select element { 'before1' : blnBefore1, 'before2' : blnBefore2, 'after1' : blnAfter1, 'after2' : blnAfter2, 'meet1' : blnMeet1, 'meet2' : blnMeet2, 'metby1' : blnMetBy1, 'metby2' : blnMetBy2, 'overlaps1' : blnOverlaps1, 'overlaps2' : blnOverlaps2, 'overlapped1' : blnOverlapped1, 'overlapped2' : blnOverlapped2, 'overlapping1' : blnOverlapping1, 'overlapping2' : blnOverlapping2, 'starts1' : blnStarts1, 'starts2' : blnStarts2, 'startedby1' : blnStartedBy1, 'startedby2' : blnStartedBy2, 'covers1' : blnCovers1, 'covers2' : blnCovers2, 'coveredby1' : blnCoveredBy1, 'coveredby2' : blnCoveredBy2, 'ends1' : blnEnds1, 'ends2' : blnEnds2, 'endedby1' : blnEndedBy1, 'endedby2' : blnEndedBy2, 'null1': null1, 'null2': null2, 'null3': null3 }
 ;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_functions/interval_functions.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_functions/interval_functions.1.adm
index 0ba9b3e..945c5cd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_functions/interval_functions.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_functions/interval_functions.1.adm
@@ -1 +1 @@
-{ "before1": true, "before2": false, "after1": true, "after2": false, "meet1": true, "meet2": false, "metby1": true, "metby2": false, "overlaps1": true, "overlaps2": false, "overlapped1": true, "overlapped2": false, "overlap1": true, "overlap2": false, "starts1": true, "starts2": false, "startedby1": true, "startedby2": false, "covers1": true, "covers2": false, "coveredby1": true, "coveredby2": false, "ends1": true, "ends2": false, "endedby1": true, "endedby2": false, "null1": null, "null2": null, "null3": null }
+{ "before1": true, "before2": false, "after1": true, "after2": false, "meet1": true, "meet2": false, "metby1": true, "metby2": false, "overlaps1": true, "overlaps2": false, "overlapped1": true, "overlapped2": false, "overlapping1": true, "overlapping2": false, "starts1": true, "starts2": false, "startedby1": true, "startedby2": false, "covers1": true, "covers2": false, "coveredby1": true, "coveredby2": false, "ends1": true, "ends2": false, "endedby1": true, "endedby2": false, "null1": null, "null2": null, "null3": null }
\ No newline at end of file
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..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.10.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
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..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.11.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.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..c15304d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.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_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..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.13.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.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..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.14.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.15.adm
new file mode 100644
index 0000000..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.15.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.16.adm
new file mode 100644
index 0000000..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.16.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.17.adm
new file mode 100644
index 0000000..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.17.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.18.adm
new file mode 100644
index 0000000..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.18.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.19.adm
new file mode 100644
index 0000000..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.19.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.20.adm
new file mode 100644
index 0000000..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.20.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
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
new file mode 100644
index 0000000..509693a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.5.adm
@@ -0,0 +1,3 @@
+{ "staff": "Zack", "student": "Karen" }
+{ "staff": "Zack", "student": "Steve" }
+{ "staff": "Zack", "student": "Tess" }
\ No newline at end of file
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
new file mode 100644
index 0000000..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.6.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.7.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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.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..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.8.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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..f38e40b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_before/interval_before.9.adm
@@ -0,0 +1,5 @@
+{ "staff": "Franklin", "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_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..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_covers/interval_covers.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.15.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.15.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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.16.adm
new file mode 100644
index 0000000..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.16.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.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.17.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.17.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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.18.adm
new file mode 100644
index 0000000..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.18.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.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.19.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.19.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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.20.adm
new file mode 100644
index 0000000..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.20.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.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.adm
new file mode 100644
index 0000000..4e22101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.5.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.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.6.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.6.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.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..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..1a67e10
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_covers/interval_covers.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_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..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.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_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..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.11.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.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..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.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_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..5af9101
--- /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.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.15.adm
new file mode 100644
index 0000000..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.15.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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.16.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.16.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.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.17.adm
new file mode 100644
index 0000000..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.17.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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.18.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.18.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.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.19.adm
new file mode 100644
index 0000000..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.19.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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.20.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.20.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.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.5.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.5.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.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.6.adm
new file mode 100644
index 0000000..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.6.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.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..5af9101
--- /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..5af9101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.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_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..5857139
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_ends/interval_ends.9.adm
@@ -0,0 +1 @@
+{ "staff": "Zack", "student": "Olga" }
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..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.10.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
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..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.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_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..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.12.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
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..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.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_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..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.14.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.15.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.15.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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.16.adm
new file mode 100644
index 0000000..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.16.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.17.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.17.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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.18.adm
new file mode 100644
index 0000000..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.18.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.19.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.19.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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.20.adm
new file mode 100644
index 0000000..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.20.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "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
new file mode 100644
index 0000000..5568195
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.5.adm
@@ -0,0 +1,2 @@
+{ "staff": "Alex", "student": "Tess" }
+{ "staff": "Henry", "student": "Tess" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.6.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.6.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.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..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.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_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..cd4b3ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_meets/interval_meets.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_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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.10.adm
@@ -0,0 +1,35 @@
+{ "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": "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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.11.adm
@@ -0,0 +1,35 @@
+{ "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": "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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.12.adm
@@ -0,0 +1,35 @@
+{ "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": "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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.13.adm
@@ -0,0 +1,35 @@
+{ "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": "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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.14.adm
@@ -0,0 +1,35 @@
+{ "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": "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.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm
new file mode 100644
index 0000000..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.5.adm
@@ -0,0 +1,35 @@
+{ "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": "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.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm
new file mode 100644
index 0000000..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.6.adm
@@ -0,0 +1,35 @@
+{ "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": "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.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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.7.adm
@@ -0,0 +1,35 @@
+{ "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": "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..4ecd143
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlapping/interval_overlapping.9.adm
@@ -0,0 +1,35 @@
+{ "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": "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..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.10.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
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..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.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_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..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.12.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
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..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.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_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..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.14.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.15.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.15.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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.16.adm
new file mode 100644
index 0000000..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.16.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.17.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.17.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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.18.adm
new file mode 100644
index 0000000..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.18.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.19.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.19.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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.20.adm
new file mode 100644
index 0000000..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.20.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "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
new file mode 100644
index 0000000..94ac56a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
@@ -0,0 +1,6 @@
+{ "staff": "Alex", "student": "Karen" }
+{ "staff": "Alex", "student": "Steve" }
+{ "staff": "Franklin", "student": "Steve" }
+{ "staff": "Franklin", "student": "Tess" }
+{ "staff": "Henry", "student": "Karen" }
+{ "staff": "Henry", "student": "Steve" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.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.6.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.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..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.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_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..9e5549f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.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_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..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.10.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.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..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.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_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..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.12.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.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..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.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_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..54ab54e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.14.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.15.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.15.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.15.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.16.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.16.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.16.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.17.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.17.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.17.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.18.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.18.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.18.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.19.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.19.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.19.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.20.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.20.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.20.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.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.5.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.5.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.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.6.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.6.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..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.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_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..6cd921a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_starts/interval_starts.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_parser_sqlpp/temporal/interval_functions/interval_functions.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/interval_functions/interval_functions.3.ast
index c6dd653..5fba565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/interval_functions/interval_functions.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/interval_functions/interval_functions.3.ast
@@ -162,13 +162,13 @@
     Variable [ Name=$itv8 ]
     Variable [ Name=$itv9 ]
   ]
-Let Variable [ Name=$blnOverlap1 ]
+Let Variable [ Name=$blnOverlapping1 ]
   :=
   FunctionCall test.interval-overlapping@2[
     Variable [ Name=$itv9 ]
     Variable [ Name=$itv7 ]
   ]
-Let Variable [ Name=$blnOverlap2 ]
+Let Variable [ Name=$blnOverlapping2 ]
   :=
   FunctionCall test.interval-overlapping@2[
     Variable [ Name=$itv9 ]
@@ -377,14 +377,14 @@
     Variable [ Name=$blnOverlapped2 ]
   )
   (
-    LiteralExpr [STRING] [overlap1]
+    LiteralExpr [STRING] [overlapping1]
     :
-    Variable [ Name=$blnOverlap1 ]
+    Variable [ Name=$blnOverlapping1 ]
   )
   (
-    LiteralExpr [STRING] [overlap2]
+    LiteralExpr [STRING] [overlapping2]
     :
-    Variable [ Name=$blnOverlap2 ]
+    Variable [ Name=$blnOverlapping2 ]
   )
   (
     LiteralExpr [STRING] [starts1]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index 85884f8..352e0d1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -22,6 +22,7 @@
              <!ENTITY APIQueries SYSTEM "queries/api/APIQueries.xml">
              <!ENTITY ComparisonQueries SYSTEM "queries/comparison/ComparisonQueries.xml">
              <!ENTITY DeepEqualQueries SYSTEM "queries/comparison/deep_equal/DeepEqualQueries.xml">
+             <!ENTITY APIQueries SYSTEM "queries/api/APIQueries.xml">
              <!ENTITY RecordsQueries SYSTEM "queries/records/RecordsQueries.xml">
              <!ENTITY TemporalQueries SYSTEM "queries/temporal/TemporalQueries.xml">
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java
new file mode 100644
index 0000000..e638de9
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractExpressionAnnotation;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+
+public class IntervalJoinExpressionAnnotation extends AbstractExpressionAnnotation {
+
+    private static final String RAW_HINT_STRING = "interval-raw-join";
+    private static final String PARTITION_HINT_STRING = "interval-partition-join";
+    private static final String MERGE_HINT_STRING = "interval-merge-join";
+    private static final String SPATIAL_HINT_STRING = "interval-spatial-join";
+    private static final String INDEX_HINT_STRING = "interval-index-join";
+    public static final IntervalJoinExpressionAnnotation INSTANCE = new IntervalJoinExpressionAnnotation();
+
+    private IRangeMap map = null;
+    private String joinType = null;
+    private long leftMaxDuration = -1;
+    private long rightMaxDuration = -1;
+    private long leftRecordCount = -1;
+    private long rightRecordCount = -1;
+    private int tuplesPerFrame = -1;
+
+    @Override
+    public IExpressionAnnotation copy() {
+        IntervalJoinExpressionAnnotation clone = new IntervalJoinExpressionAnnotation();
+        clone.setObject(object);
+        return clone;
+    }
+
+    @Override
+    public void setObject(Object object) {
+        super.setObject(object);
+        parseHint();
+    }
+
+    private void parseHint() {
+        String hint = (String) object;
+        setJoinType(hint);
+
+        if (joinType.equals(PARTITION_HINT_STRING)) {
+            hint = hint.substring(hint.indexOf(']', 0) + 1).trim();
+            String[] args = hint.split(" ");
+            if (args.length == 5) {
+                leftRecordCount = Long.valueOf(args[0]);
+                rightRecordCount = Long.valueOf(args[1]);
+                leftMaxDuration = Long.valueOf(args[2]);
+                rightMaxDuration = Long.valueOf(args[3]);
+                tuplesPerFrame = Integer.valueOf(args[4]);
+            }
+        }
+    }
+
+    private void setJoinType(String hint) {
+        if (hint.startsWith(RAW_HINT_STRING)) {
+            joinType = RAW_HINT_STRING;
+        } else if (hint.startsWith(PARTITION_HINT_STRING)) {
+            joinType = PARTITION_HINT_STRING;
+        } else if (hint.startsWith(MERGE_HINT_STRING)) {
+            joinType = MERGE_HINT_STRING;
+        } else if (hint.startsWith(SPATIAL_HINT_STRING)) {
+            joinType = SPATIAL_HINT_STRING;
+        } else if (hint.startsWith(INDEX_HINT_STRING)) {
+            joinType = INDEX_HINT_STRING;
+        }
+    }
+
+    public long getLeftMaxDuration() {
+        return leftMaxDuration;
+    }
+
+    public long getRightMaxDuration() {
+        return rightMaxDuration;
+    }
+
+    public long getLeftRecordCount() {
+        return leftRecordCount;
+    }
+
+    public long getRightRecordCount() {
+        return rightRecordCount;
+    }
+
+    public int getTuplesPerFrame() {
+        return tuplesPerFrame;
+    }
+
+    public void setRangeMap(IRangeMap map) {
+        this.map = map;
+    }
+
+    public IRangeMap getRangeMap() {
+        return map;
+    }
+
+    public String getRangeType() {
+        return joinType;
+    }
+
+    public boolean hasRangeArgument() {
+        if (joinType.equals(RAW_HINT_STRING)) {
+            return false;
+        }
+        return true;
+    }
+
+    public boolean isRawJoin() {
+        if (joinType.equals(RAW_HINT_STRING)) {
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isPartitionJoin() {
+        if (joinType.equals(PARTITION_HINT_STRING)) {
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isMergeJoin() {
+        if (joinType.equals(MERGE_HINT_STRING)) {
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isSpatialJoin() {
+        if (joinType.equals(SPATIAL_HINT_STRING)) {
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isIndexJoin() {
+        if (joinType.equals(INDEX_HINT_STRING)) {
+            return true;
+        }
+        return false;
+    }
+
+    public static boolean isIntervalJoinHint(String hint) {
+        return hint.startsWith(RAW_HINT_STRING) || hint.startsWith(PARTITION_HINT_STRING)
+                || hint.startsWith(MERGE_HINT_STRING) || hint.startsWith(SPATIAL_HINT_STRING)
+                || hint.startsWith(INDEX_HINT_STRING);
+    }
+
+    public static int getHintLength(String hint) {
+        if (hint.startsWith(RAW_HINT_STRING)) {
+            return RAW_HINT_STRING.length();
+        } else if (hint.startsWith(PARTITION_HINT_STRING)) {
+            return PARTITION_HINT_STRING.length();
+        } else if (hint.startsWith(MERGE_HINT_STRING)) {
+            return MERGE_HINT_STRING.length();
+        } else if (hint.startsWith(SPATIAL_HINT_STRING)) {
+            return SPATIAL_HINT_STRING.length();
+        } else if (hint.startsWith(INDEX_HINT_STRING)) {
+            return INDEX_HINT_STRING.length();
+        }
+        return 0;
+    }
+
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
index af5124b..232a541 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixCompilerProperties.java
@@ -24,6 +24,8 @@
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
 public class AsterixCompilerProperties extends AbstractAsterixProperties {
+    private static final int MB = 1048576;
+
     private static final String COMPILER_SORTMEMORY_KEY = "compiler.sortmemory";
     private static final long COMPILER_SORTMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
@@ -33,9 +35,15 @@
     private static final String COMPILER_JOINMEMORY_KEY = "compiler.joinmemory";
     private static final long COMPILER_JOINMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
+    private static final String COMPILER_INTERVAL_MAXDURATION_KEY = "compiler.interval.maxduration";
+    private static final long COMPILER_INTERVAL_MAXDURATION_DEFAULT = 1000;
+
     private static final String COMPILER_FRAMESIZE_KEY = "compiler.framesize";
     private static final int COMPILER_FRAMESIZE_DEFAULT = StorageUtil.getSizeInBytes(32, KILOBYTE);
 
+    private static final String COMPILER_JOIN_LEFTINPUT_KEY = "compiler.join.leftinput";
+    private static final long COMPILER_JOIN_LEFTINPUT_DEFAULT = (int) ((140L * 1024 * MB) / COMPILER_FRAMESIZE_DEFAULT); // 140GB
+
     private static final String COMPILER_PREGELIX_HOME = "compiler.pregelix.home";
     private static final String COMPILER_PREGELIX_HOME_DEFAULT = "~/pregelix";
 
@@ -55,12 +63,21 @@
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
-    @PropertyKey(COMPILER_GROUPMEMORY_KEY)
+    public long getJoinLeftInputSize() {
+        return accessor.getProperty(COMPILER_JOIN_LEFTINPUT_KEY, COMPILER_JOIN_LEFTINPUT_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
     public long getGroupMemorySize() {
         return accessor.getProperty(COMPILER_GROUPMEMORY_KEY, COMPILER_GROUPMEMORY_DEFAULT,
                 PropertyInterpreters.getLongBytePropertyInterpreter());
     }
 
+    public long getIntervalMaxDuration() {
+        return accessor.getProperty(COMPILER_INTERVAL_MAXDURATION_KEY, COMPILER_INTERVAL_MAXDURATION_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
     @PropertyKey(COMPILER_FRAMESIZE_KEY)
     public int getFrameSize() {
         return accessor.getProperty(COMPILER_FRAMESIZE_KEY, COMPILER_FRAMESIZE_DEFAULT,
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
index 2ee70bc..6d86976 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
@@ -49,15 +49,18 @@
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
 import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public abstract class RangeMapBuilder {
     private static final IParserFactory parserFactory = new AQLParserFactory();
 
+    private RangeMapBuilder() {
+    }
+
     public static IRangeMap parseHint(Object hint) throws AsterixException {
         ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
         DataOutput out = abvs.getDataOutput();;
@@ -65,7 +68,9 @@
 
         IParser parser = parserFactory.createParser((String) hint);
         List<Statement> hintStatements = parser.parse();
-        if (hintStatements.size() != 1) {
+        if (hintStatements.isEmpty()) {
+            throw new AsterixException("No range hint was supplied to the RangeMapBuilder.");
+        } else if (hintStatements.size() != 1) {
             throw new AsterixException("Only one range statement is allowed for the range hint.");
         }
 
@@ -79,7 +84,7 @@
             throw new AsterixException("The range hint must be a list.");
         }
         List<Expression> el = ((ListConstructor) q.getBody()).getExprList();
-        int offsets[] = new int[el.size()];
+        int[] offsets = new int[el.size()];
 
         // Loop over list of literals
         for (int i = 0; i < el.size(); ++i) {
@@ -154,7 +159,7 @@
         IBinaryComparatorFactory bcf =
                 comparatorFactory.getBinaryComparatorFactory(ATypeTag.VALUE_TYPE_MAPPING[fieldType], ascending);
         IBinaryComparator comparator = bcf.createBinaryComparator();
-        int c = 0;
+        int c;
         for (int split = 1; split < rangeMap.getSplitCount(); ++split) {
             if (fieldType != rangeMap.getTag(fieldIndex, split)) {
                 throw new AsterixException("Range field contains more than a single type of items (" + fieldType
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 0a9079d..0dfe832 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -36,6 +36,7 @@
 import org.apache.asterix.common.annotations.FieldValFileSameIndexDataGen;
 import org.apache.asterix.common.annotations.IRecordFieldDataGen;
 import org.apache.asterix.common.annotations.InsertRandIntDataGen;
+import org.apache.asterix.common.annotations.IntervalJoinExpressionAnnotation;
 import org.apache.asterix.common.annotations.ListDataGen;
 import org.apache.asterix.common.annotations.ListValFileDataGen;
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
@@ -147,7 +148,7 @@
     private static final String DATETIME_ADD_RAND_HOURS_HINT = "datetime-add-rand-hours";
     private static final String DATETIME_BETWEEN_YEARS_HINT = "datetime-between-years";
     private static final String HASH_GROUP_BY_HINT = "hash";
-    private static final String INDEXED_NESTED_LOOP_JOIN_HINT = "indexnl";
+    private static final String INDEXED_NESTED_LOOP_JOIN_HINT = IndexedNLJoinExpressionAnnotation.HINT_STRING;
     private static final String INMEMORY_HINT = "inmem";
     private static final String INSERT_RAND_INT_HINT = "insert-rand-int";
     private static final String INTERVAL_HINT = "interval";
@@ -2175,7 +2176,19 @@
       }
       callExpr = new CallExpr(signature,argList);
       if (hint != null) {
-        if (hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
+        if (IntervalJoinExpressionAnnotation.isIntervalJoinHint(hint)) {
+          IntervalJoinExpressionAnnotation ijea = IntervalJoinExpressionAnnotation.INSTANCE;
+          ijea.setObject(hint);
+          try {
+            if (ijea.hasRangeArgument()) {
+              String rangeHint = hint.substring(IntervalJoinExpressionAnnotation.getHintLength(hint), hint.indexOf(']', 0) + 1);
+              ijea.setRangeMap(RangeMapBuilder.parseHint(rangeHint));
+            }
+          } catch (AsterixException e) {
+            {if (true) throw new ParseException(e.getMessage());}
+          }
+          callExpr.addHint(ijea);
+        } else if (hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
           callExpr.addHint(IndexedNLJoinExpressionAnnotation.INSTANCE);
         } else if (hint.startsWith(SKIP_SECONDARY_INDEX_SEARCH_HINT)) {
           callExpr.addHint(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE);
@@ -2367,7 +2380,7 @@
           }
           if (hint.startsWith(RANGE_HINT)) {
             try{
-              oc.setRangeMap(RangeMapBuilder.parseHint(hint.substring(RANGE_HINT.length())));
+              oc.setRangeMap(RangeMapBuilder.parseHint(hint.substring(RANGE_HINT.length(), hint.indexOf(']', 0) + 1)));
             } catch (AsterixException e) {
               throw new ParseException(e.getMessage());
             }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index 8574c76..4c66441 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -25,7 +25,7 @@
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.commons.lang3.ObjectUtils;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
 
 public class OrderbyClause implements Clause {
     private List<Expression> orderbyList;
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index 72bf65e..81bd54d 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -64,6 +64,12 @@
       <artifactId>hyracks-storage-am-lsm-rtree</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-test-support</artifactId>
+      <type>jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.reflections</groupId>
       <artifactId>reflections</artifactId>
       <version>0.9.10</version>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscRangeBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscRangeBinaryComparatorFactory.java
new file mode 100644
index 0000000..c51c6da
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectAscRangeBinaryComparatorFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.comparators;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+
+public class AObjectAscRangeBinaryComparatorFactory implements IBinaryRangeComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AObjectAscRangeBinaryComparatorFactory INSTANCE = new AObjectAscRangeBinaryComparatorFactory();
+
+    private AObjectAscRangeBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        return AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescRangeBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescRangeBinaryComparatorFactory.java
new file mode 100644
index 0000000..54199d8
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AObjectDescRangeBinaryComparatorFactory.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.comparators;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+
+public class AObjectDescRangeBinaryComparatorFactory implements IBinaryRangeComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AObjectDescRangeBinaryComparatorFactory INSTANCE = new AObjectDescRangeBinaryComparatorFactory();
+
+    private AObjectDescRangeBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        return AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalAscRangeBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalAscRangeBinaryComparatorFactory.java
new file mode 100644
index 0000000..121204f
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalAscRangeBinaryComparatorFactory.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.dataflow.data.nontagged.comparators.rangeinterval;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+
+public class IntervalAscRangeBinaryComparatorFactory implements IBinaryRangeComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IBinaryRangeComparatorFactory INSTANCE = new IntervalAscRangeBinaryComparatorFactory();
+
+    private IntervalAscRangeBinaryComparatorFactory() {
+
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return Long.compare(AIntervalSerializerDeserializer.getIntervalStart(b1, s1),
+                        AInt64SerializerDeserializer.getLong(b2, s2));
+            }
+        };
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return Long.compare(AIntervalSerializerDeserializer.getIntervalEnd(b1, s1),
+                        AInt64SerializerDeserializer.getLong(b2, s2));
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalDescRangeBinaryComparatorFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalDescRangeBinaryComparatorFactory.java
new file mode 100644
index 0000000..fc14dca
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalDescRangeBinaryComparatorFactory.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.dataflow.data.nontagged.comparators.rangeinterval;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+
+public class IntervalDescRangeBinaryComparatorFactory implements IBinaryRangeComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IBinaryRangeComparatorFactory INSTANCE = new IntervalDescRangeBinaryComparatorFactory();
+
+    private IntervalDescRangeBinaryComparatorFactory() {
+
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return -Long.compare(AIntervalSerializerDeserializer.getIntervalEnd(b1, s1),
+                        AInt64SerializerDeserializer.getLong(b2, s2));
+            }
+        };
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return -Long.compare(AIntervalSerializerDeserializer.getIntervalStart(b1, s1),
+                        AInt64SerializerDeserializer.getLong(b2, s2));
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
index a8c6eea..d2c232d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
@@ -25,7 +25,9 @@
 import org.apache.asterix.dataflow.data.nontagged.comparators.AIntervalDescPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscRangeBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescRangeBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
@@ -33,11 +35,15 @@
 import org.apache.asterix.dataflow.data.nontagged.comparators.AUUIDPartialBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.BooleanBinaryComparatorFactory;
 import org.apache.asterix.dataflow.data.nontagged.comparators.RawBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.rangeinterval.IntervalAscRangeBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.rangeinterval.IntervalDescRangeBinaryComparatorFactory;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.IAType;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
 import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
@@ -79,6 +85,21 @@
     private AqlBinaryComparatorFactoryProvider() {
     }
 
+    // This method adds the option of range range
+    @Override
+    public IBinaryRangeComparatorFactory getRangeBinaryComparatorFactory(Object type, boolean ascending,
+            RangePartitioningType rangeType) {
+        if (type == null) {
+            return anyBinaryRangeComparatorFactory(ascending);
+        }
+        IAType aqlType = (IAType) type;
+        if (aqlType.getTypeTag() == ATypeTag.INTERVAL) {
+            return addOffsetForRange(getIntervalRangeBinaryComparatorFactory(ascending), ascending);
+        } else {
+            return anyBinaryRangeComparatorFactory(ascending);
+        }
+    }
+
     // This method add the option of ignoring the case in string comparisons.
     // TODO: We should incorporate this option more nicely, but I'd have to change algebricks.
     public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending, boolean ignoreCase) {
@@ -173,7 +194,7 @@
                 return addOffset(ADurationPartialBinaryComparatorFactory.INSTANCE, ascending);
             }
             case INTERVAL: {
-                return addOffset(intervalBinaryComparatorFactory(ascending), ascending);
+                return addOffset(getIntervalBinaryComparatorFactory(ascending), ascending);
             }
             case UUID: {
                 return addOffset(AUUIDPartialBinaryComparatorFactory.INSTANCE, ascending);
@@ -197,7 +218,58 @@
                 final IBinaryComparator bc = inst.createBinaryComparator();
                 if (ascending) {
                     return new ABinaryComparator() {
+                        @Override
+                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
+                                throws HyracksDataException {
+                            return bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                        }
+                    };
+                } else {
+                    return new ABinaryComparator() {
+                        @Override
+                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
+                                throws HyracksDataException {
+                            return -bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                        }
+                    };
+                }
+            }
+        };
+    }
 
+    private IBinaryRangeComparatorFactory addOffsetForRange(final IBinaryRangeComparatorFactory inst,
+            final boolean ascending) {
+        return new IBinaryRangeComparatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IBinaryComparator createMinBinaryComparator() {
+                final IBinaryComparator bc = inst.createMinBinaryComparator();
+                if (ascending) {
+                    return new ABinaryComparator() {
+                        @Override
+                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
+                                throws HyracksDataException {
+                            return bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                        }
+                    };
+                } else {
+                    return new ABinaryComparator() {
+                        @Override
+                        public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
+                                throws HyracksDataException {
+                            return -bc.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                        }
+                    };
+                }
+            }
+
+            @Override
+            public IBinaryComparator createMaxBinaryComparator() {
+                final IBinaryComparator bc = inst.createMaxBinaryComparator();
+                if (ascending) {
+                    return new ABinaryComparator() {
                         @Override
                         public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2)
                                 throws HyracksDataException {
@@ -225,7 +297,23 @@
         }
     }
 
-    private IBinaryComparatorFactory intervalBinaryComparatorFactory(boolean ascending) {
+    private IBinaryRangeComparatorFactory anyBinaryRangeComparatorFactory(boolean ascending) {
+        if (ascending) {
+            return AObjectAscRangeBinaryComparatorFactory.INSTANCE;
+        } else {
+            return AObjectDescRangeBinaryComparatorFactory.INSTANCE;
+        }
+    }
+
+    private IBinaryRangeComparatorFactory getIntervalRangeBinaryComparatorFactory(boolean ascending) {
+        if (ascending) {
+            return IntervalAscRangeBinaryComparatorFactory.INSTANCE;
+        } else {
+            return IntervalDescRangeBinaryComparatorFactory.INSTANCE;
+        }
+    }
+
+    private IBinaryComparatorFactory getIntervalBinaryComparatorFactory(boolean ascending) {
         // Intervals have separate binary comparator factories, since asc is primarily based on start point
         // and desc is similarly based on end point.
         if (ascending) {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
index 873ea9e..4de9456 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -585,6 +585,10 @@
             "interval-ends", 2);
     public static final FunctionIdentifier INTERVAL_ENDED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "interval-ended-by", 2);
+    public static final FunctionIdentifier INTERVAL_PARTITION_JOIN_START = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "interval-partition-join-start", 3);
+    public static final FunctionIdentifier INTERVAL_PARTITION_JOIN_END = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "interval-partition-join-end", 3);
     public static final FunctionIdentifier CURRENT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "current-time", 0);
     public static final FunctionIdentifier CURRENT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
@@ -1110,6 +1114,8 @@
         addFunction(INTERVAL_COVERED_BY, ABooleanTypeComputer.INSTANCE, true);
         addFunction(INTERVAL_ENDS, ABooleanTypeComputer.INSTANCE, true);
         addFunction(INTERVAL_ENDED_BY, ABooleanTypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERVAL_PARTITION_JOIN_START, AInt32TypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERVAL_PARTITION_JOIN_END, AInt32TypeComputer.INSTANCE, true);
         addFunction(CURRENT_DATE, ADateTypeComputer.INSTANCE, false);
         addFunction(CURRENT_TIME, ATimeTypeComputer.INSTANCE, false);
         addFunction(CURRENT_DATETIME, ADateTimeTypeComputer.INSTANCE, false);
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/AqlBinaryComparatorFactoryTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/AqlBinaryComparatorFactoryTest.java
new file mode 100644
index 0000000..a6c449f
--- /dev/null
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/AqlBinaryComparatorFactoryTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged.comparators;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInterval;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class AqlBinaryComparatorFactoryTest extends TestCase {
+
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer serde = AObjectSerializerDeserializer.INSTANCE;
+
+    /*
+     * The following points (X) will be tested for this interval (+).
+     *
+     * ----X---XXX---X---XXX---X----
+     * ---------+++++++++++---------
+     */
+    private final int INTERVAL_LENGTH = Byte.BYTES + Byte.BYTES + Long.BYTES + Long.BYTES;
+    private final int INTEGER_LENGTH = Byte.BYTES + Long.BYTES;
+    private final AInterval[] INTERVALS = new AInterval[] { new AInterval(10, 15, (byte) 16),
+            new AInterval(10, 20, (byte) 16), new AInterval(15, 20, (byte) 16) };
+    private final AInt64[] INTEGERS = new AInt64[] { new AInt64(10l), new AInt64(15l), new AInt64(20l) };
+
+    @SuppressWarnings("unused")
+    private byte[] getIntervalBytes() throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < INTERVALS.length; ++i) {
+                serde.serialize(INTERVALS[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private byte[] getIntegerBytes() throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < INTEGERS.length; ++i) {
+                serde.serialize(INTEGERS[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private void executeBinaryComparatorTests(IBinaryComparator bc, byte[] bytes, int fieldLength, int[][] results)
+            throws HyracksDataException {
+        for (int i = 0; i < results.length; ++i) {
+            int leftOffset = i * fieldLength;
+            for (int j = 0; j < results[i].length; ++j) {
+                int rightOffset = j * fieldLength;
+                int c = bc.compare(bytes, leftOffset, fieldLength, bytes, rightOffset, fieldLength);
+                Assert.assertEquals("results[" + i + "][" + j + "]", results[i][j], c);
+            }
+        }
+    }
+
+    //    private final AInterval[] INTERVALS = new AInterval[] { new AInterval(10, 20, (byte) 16),
+    //            new AInterval(10, 15, (byte) 16), new AInterval(15, 20, (byte) 16) };
+    //    private final AInt64[] INTEGERS = new AInt64[] { new AInt64(10l), new AInt64(15l), new AInt64(20l) };
+    //
+    @Test
+    public void testIntervalAsc() throws HyracksDataException {
+        IBinaryComparator bc = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        byte[] bytes = getIntervalBytes();
+        int[][] results = new int[3][];
+        results[0] = new int[] { 0, -1, -1 };
+        results[1] = new int[] { 1, 0, -1 };
+        results[2] = new int[] { 1, 1, 0 };
+        executeBinaryComparatorTests(bc, bytes, INTERVAL_LENGTH, results);
+    }
+
+    @Test
+    public void testIntervalDesc() throws HyracksDataException {
+        IBinaryComparator bc = AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        byte[] bytes = getIntervalBytes();
+        int[][] results = new int[3][];
+        results[0] = new int[] { 0, 1, 1 };
+        results[1] = new int[] { -1, 0, 1 };
+        results[2] = new int[] { -1, -1, 0 };
+        executeBinaryComparatorTests(bc, bytes, INTERVAL_LENGTH, results);
+    }
+
+    @Test
+    public void testIntegerAsc() throws HyracksDataException {
+        IBinaryComparator bc = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        byte[] bytes = getIntegerBytes();
+        int[][] results = new int[3][];
+        results[0] = new int[] { 0, -1, -1 };
+        results[1] = new int[] { 1, 0, -1 };
+        results[2] = new int[] { 1, 1, 0 };
+        executeBinaryComparatorTests(bc, bytes, INTEGER_LENGTH, results);
+    }
+
+    @Test
+    public void testIngeterDesc() throws HyracksDataException {
+        IBinaryComparator bc = AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+        byte[] bytes = getIntegerBytes();
+        int[][] results = new int[3][];
+        results[0] = new int[] { 0, 1, 1 };
+        results[1] = new int[] { -1, 0, 1 };
+        results[2] = new int[] { -1, -1, 0 };
+        executeBinaryComparatorTests(bc, bytes, INTEGER_LENGTH, results);
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalRangePartitionComputerFactoryTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalRangePartitionComputerFactoryTest.java
new file mode 100644
index 0000000..2179634
--- /dev/null
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/IntervalRangePartitionComputerFactoryTest.java
@@ -0,0 +1,321 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged.comparators.rangeinterval;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.asterix.om.base.AInterval;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.storage.IGrowableIntArray;
+import org.apache.hyracks.dataflow.common.comm.io.FrameFixedFieldTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.storage.common.arraylist.IntArrayList;
+import org.apache.hyracks.test.support.TestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class IntervalRangePartitionComputerFactoryTest extends TestCase {
+
+    private final ISerializerDeserializer<AInterval> intervalSerde = AIntervalSerializerDeserializer.INSTANCE;
+    private final Integer64SerializerDeserializer int64Serde = Integer64SerializerDeserializer.INSTANCE;
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] {
+            AIntervalSerializerDeserializer.INSTANCE };
+    private final RecordDescriptor RecordDesc = new RecordDescriptor(SerDers);
+
+    IBinaryRangeComparatorFactory[] BINARY_ASC_COMPARATOR_FACTORIES = new IBinaryRangeComparatorFactory[] {
+            IntervalAscRangeBinaryComparatorFactory.INSTANCE };
+    IBinaryRangeComparatorFactory[] BINARY_DESC_COMPARATOR_FACTORIES = new IBinaryRangeComparatorFactory[] {
+            IntervalDescRangeBinaryComparatorFactory.INSTANCE };
+    /*
+     * The following three intervals (+++) will be tested for these 4 partitions.
+     *
+     *    ----------+++++++++++++++++++++++++++----------
+     *    -----------+++++++++++++++++++++++++-----------
+     *    ------------+++++++++++++++++++++++------------
+     *    -----------|-----------|-----------|-----------
+     *     or 16 partitions.
+     *    --|--|--|--|--|--|--|--|--|--|--|--|--|--|--|--
+     */
+
+    private final int FRAME_SIZE = 320;
+    private final int INTEGER_LENGTH = Long.BYTES;
+
+    private final AInterval[] INTERVALS = new AInterval[] { new AInterval(99, 301, (byte) 16),
+            new AInterval(100, 300, (byte) 16), new AInterval(101, 299, (byte) 16) };
+    private final int INTERVAL_LENGTH = Byte.BYTES + Long.BYTES + Long.BYTES;
+
+    //map          { 0l, 25l, 50l, 75l, 100l, 125l, 150l, 175l, 200l, 225l, 250l, 275l, 300l, 325l, 350l, 375l, 400l };
+    //partitions   {    0,   1,   2,   3,    4,    5,    6,    7,    8,    9,    10,   11,   12,   13,   14,  15     };
+    private final Long[] MAP_POINTS = new Long[] { 0l, 25l, 50l, 75l, 100l, 125l, 150l, 175l, 200l, 225l, 250l, 275l,
+            300l, 325l, 350l, 375l, 400l };
+
+    @SuppressWarnings("unused")
+    private byte[] getIntervalBytes(AInterval[] intervals) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < intervals.length; ++i) {
+                intervalSerde.serialize(intervals[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private byte[] getIntegerBytes(Long[] integers) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < integers.length; ++i) {
+                int64Serde.serialize(integers[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private IRangeMap getRangeMap(Long[] integers) throws HyracksDataException {
+        int offsets[] = new int[integers.length];
+        for (int i = 0; i < integers.length; ++i) {
+            offsets[i] = (i + 1) * INTEGER_LENGTH;
+        }
+        return new RangeMap(1, getIntegerBytes(integers), offsets);
+    }
+
+    private ByteBuffer prepareData(IHyracksTaskContext ctx, AInterval[] intervals) throws HyracksDataException {
+        IFrame frame = new VSizeFrame(ctx);
+        FrameFixedFieldTupleAppender fffta = new FrameFixedFieldTupleAppender(RecordDesc.getFieldCount());
+        fffta.reset(frame, true);
+
+        byte[] serializedIntervals = getIntervalBytes(intervals);
+        for (int i = 0; i < intervals.length; ++i) {
+            fffta.appendField(serializedIntervals, i * INTERVAL_LENGTH, INTERVAL_LENGTH);
+        }
+
+        return frame.getBuffer();
+    }
+
+    private void executeFieldRangePartitionTests(AInterval[] intervals, IRangeMap rangeMap,
+            IBinaryRangeComparatorFactory[] comparatorFactories, RangePartitioningType rangeType, int nParts,
+            int[][] results) throws HyracksDataException {
+        IHyracksTaskContext ctx = TestUtils.create(FRAME_SIZE);
+        int[] rangeFields = new int[] { 0 };
+        ITupleRangePartitionComputerFactory frpcf = new FieldRangePartitionComputerFactory(rangeFields,
+                comparatorFactories, rangeType);
+        ITupleRangePartitionComputer partitioner = frpcf.createPartitioner(rangeMap);
+
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(RecordDesc);
+        ByteBuffer buffer = prepareData(ctx, intervals);
+        accessor.reset(buffer);
+
+        IGrowableIntArray map = new IntArrayList(16, 1);
+
+        for (int i = 0; i < results.length; ++i) {
+            map.clear();
+            partitioner.partition(accessor, i, nParts, map);
+            checkPartitionResult(intervals[i], results[i], map);
+        }
+    }
+
+    private String getString(int[] results) {
+        String result = "[";
+        for (int i = 0; i < results.length; ++i) {
+            result += results[i];
+            if (i < results.length - 1) {
+                result += ", ";
+            }
+        }
+        result += "]";
+        return result;
+    }
+
+    private String getString(IGrowableIntArray results) {
+        String result = "[";
+        for (int i = 0; i < results.size(); ++i) {
+            result += results.get(i);
+            if (i < results.size() - 1) {
+                result += ", ";
+            }
+        }
+        result += "]";
+        return result;
+    }
+
+    private void checkPartitionResult(AInterval value, int[] results, IGrowableIntArray map) {
+        if (results.length != map.size()) {
+            Assert.assertEquals("The partition for value (" + value + ") gives different number of partitions",
+                    results.length, map.size());
+        }
+        for (int i = 0; i < results.length; ++i) {
+            boolean match = false;
+            for (int j = 0; j < results.length; ++j) {
+                if (results[i] == map.get(j)) {
+                    match = true;
+                    continue;
+                }
+            }
+            if (!match) {
+                Assert.assertEquals("Individual partitions for " + value + " do not match", getString(results),
+                        getString(map));
+                return;
+            }
+        }
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProject16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3 };
+        results[1] = new int[] { 4 };
+        results[2] = new int[] { 4 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescProject16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3 };
+        results[1] = new int[] { 4 };
+        results[2] = new int[] { 4 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProjectEnd16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 12 };
+        results[1] = new int[] { 12 };
+        results[2] = new int[] { 11 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT_END, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescProjectEnd16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 12 };
+        results[1] = new int[] { 12 };
+        results[2] = new int[] { 11 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT_END, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscSplit16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 };
+        results[1] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12 };
+        results[2] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.SPLIT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescSplit16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 };
+        results[1] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12 };
+        results[2] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.SPLIT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscReplicate16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[1] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[2] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.REPLICATE, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescReplicate16Partitions() throws HyracksDataException {
+        int[][] results = new int[3][];
+        results[0] = new int[] { 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[1] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[2] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(INTERVALS, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.REPLICATE, 16, results);
+    }
+
+}
diff --git a/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalBinaryComparatorFactoryTest.java b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalBinaryComparatorFactoryTest.java
new file mode 100644
index 0000000..d8fd8d6
--- /dev/null
+++ b/asterixdb/asterix-om/src/test/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalBinaryComparatorFactoryTest.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.dataflow.data.nontagged.comparators.rangeinterval;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AInterval;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class RangeIntervalBinaryComparatorFactoryTest extends TestCase {
+
+    private final ISerializerDeserializer<AInterval> intervalSerde = AIntervalSerializerDeserializer.INSTANCE;
+    private final ISerializerDeserializer<AInt64> intSerde = AInt64SerializerDeserializer.INSTANCE;
+
+    /*
+     * The following points (X) will be tested for this interval (+).
+     *
+     * ----X---XXX---X---XXX---X----
+     * ---------+++++++++++---------
+     */
+    private final AInterval INTERVAL = new AInterval(10, 20, (byte) 16);
+    private final int INTERVAL_OFFSET = 0;
+    private int INTERVAL_LENGTH;
+    private final int POINT_LENGTH = Long.BYTES;
+    private final AInt64[] MAP_POINTS = new AInt64[] { new AInt64(5l), new AInt64(9l), new AInt64(10l), new AInt64(11l),
+            new AInt64(15l), new AInt64(19l), new AInt64(20l), new AInt64(21l), new AInt64(25l) };
+
+    private byte[] getIntervalBytes() throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            intervalSerde.serialize(INTERVAL, dos);
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private byte[] getIntegerMapBytes() throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < MAP_POINTS.length; ++i) {
+                intSerde.serialize(MAP_POINTS[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private void executeBinaryComparatorTests(IBinaryComparator bc, int[] results) throws HyracksDataException {
+        // Bytes for interval
+        byte[] intervalBytes = getIntervalBytes();
+        INTERVAL_LENGTH = AIntervalSerializerDeserializer.getIntervalLength(intervalBytes, INTERVAL_OFFSET);
+
+        // Bytes for map points
+        byte[] intBytes = getIntegerMapBytes();
+
+        for (int i = 0; i < results.length; ++i) {
+            int point_offset = i * POINT_LENGTH;
+            int c = bc.compare(intervalBytes, INTERVAL_OFFSET, INTERVAL_LENGTH, intBytes, point_offset, POINT_LENGTH);
+            Assert.assertEquals(INTERVAL + " compares to map point (" + MAP_POINTS[i].getLongValue() + ")", results[i],
+                    c);
+        }
+    }
+
+    @Test
+    public void testAscMin() throws HyracksDataException {
+        IBinaryComparator bc = IntervalAscRangeBinaryComparatorFactory.INSTANCE.createMinBinaryComparator();
+        int[] results = new int[] { 1, 1, 0, -1, -1, -1, -1, -1, -1 };
+        executeBinaryComparatorTests(bc, results);
+    }
+
+    @Test
+    public void testAscMax() throws HyracksDataException {
+        IBinaryComparator bc = IntervalAscRangeBinaryComparatorFactory.INSTANCE.createMaxBinaryComparator();
+        int[] results = new int[] { 1, 1, 1, 1, 1, 1, 0, -1, -1 };
+        executeBinaryComparatorTests(bc, results);
+    }
+
+    @Test
+    public void testDescMin() throws HyracksDataException {
+        IBinaryComparator bc = IntervalDescRangeBinaryComparatorFactory.INSTANCE.createMinBinaryComparator();
+        int[] results = new int[] { -1, -1, -1, -1, -1, -1, 0, 1, 1 };
+        executeBinaryComparatorTests(bc, results);
+    }
+
+    @Test
+    public void testDescMax() throws HyracksDataException {
+        IBinaryComparator bc = IntervalDescRangeBinaryComparatorFactory.INSTANCE.createMaxBinaryComparator();
+        int[] results = new int[] { -1, -1, 0, 1, 1, 1, 1, 1, 1 };
+        executeBinaryComparatorTests(bc, results);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index 4224d19..6157c6e 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -63,6 +63,10 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-std</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-client</artifactId>
       <type>jar</type>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappingDescriptor.java
similarity index 92%
rename from asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappingDescriptor.java
index d27459e..58931fe 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/OverlapDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalOverlappingDescriptor.java
@@ -25,14 +25,14 @@
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class OverlapDescriptor extends AbstractIntervalLogicFuncDescriptor {
+public class IntervalOverlappingDescriptor extends AbstractIntervalLogicFuncDescriptor {
 
     private static final long serialVersionUID = 1L;
 
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new OverlapDescriptor();
+            return new IntervalOverlappingDescriptor();
         }
     };
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinEndDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinEndDescriptor.java
new file mode 100644
index 0000000..8d737f4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinEndDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class IntervalPartitionJoinEndDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new IntervalPartitionJoinEndDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+                return new IntervalPartitionJoinFunction(args, ctx, false);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.INTERVAL_PARTITION_JOIN_END;
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinFunction.java
new file mode 100644
index 0000000..4725fa5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinFunction.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.temporal;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.operators.joins.intervalpartition.IntervalPartitionUtil;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+
+public class IntervalPartitionJoinFunction implements IScalarEvaluator {
+
+    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private DataOutput out = resultStorage.getDataOutput();
+    private IPointable argPtr0 = new VoidPointable();
+    private IPointable argPtr1 = new VoidPointable();
+    private IPointable argPtr2 = new VoidPointable();
+    private int rangeIdCache = -1;
+    private long partitionStart;
+    private long partitionDuration;
+
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    private AMutableInt32 aInt = new AMutableInt32(0);
+
+    private IHyracksTaskContext ctx;
+    private IScalarEvaluator eval0;
+    private IScalarEvaluator eval1;
+    private IScalarEvaluator eval2;
+    private boolean startPoint;
+
+    public IntervalPartitionJoinFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext ctx, boolean startPoint)
+            throws AlgebricksException {
+        this.ctx = ctx;
+        this.eval0 = args[0].createScalarEvaluator(ctx);
+        this.eval1 = args[1].createScalarEvaluator(ctx);
+        this.eval2 = args[2].createScalarEvaluator(ctx);
+        this.startPoint = startPoint;
+    }
+
+    public void evaluate(IFrameTupleReference tuple, IPointable result) throws AlgebricksException {
+        resultStorage.reset();
+        // Interval
+        eval0.evaluate(tuple, argPtr0);
+        // rangeId
+        eval1.evaluate(tuple, argPtr1);
+        // k
+        eval2.evaluate(tuple, argPtr2);
+
+        byte[] bytes0 = argPtr0.getByteArray();
+        int offset0 = argPtr0.getStartOffset();
+        byte[] bytes1 = argPtr1.getByteArray();
+        int offset1 = argPtr1.getStartOffset();
+        byte[] bytes2 = argPtr2.getByteArray();
+        int offset2 = argPtr2.getStartOffset();
+
+        try {
+            if (bytes0[offset0] != ATypeTag.SERIALIZED_INTERVAL_TYPE_TAG) {
+                throw new AlgebricksException("Expected type INTERVAL for parameter 0 but got "
+                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]));
+            }
+
+            if (bytes1[offset1] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+                throw new AlgebricksException("Expected type INT32 for parameter 1 but got "
+                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]));
+            }
+
+            if (bytes2[offset2] != ATypeTag.SERIALIZED_INT32_TYPE_TAG) {
+                throw new AlgebricksException("Expected type INT32 for parameter 2 but got "
+                        + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]));
+            }
+
+            long point;
+            if (startPoint) {
+                point = AIntervalSerializerDeserializer.getIntervalStart(bytes0, offset0 + 1);
+            } else {
+                point = AIntervalSerializerDeserializer.getIntervalEnd(bytes0, offset0 + 1);
+            }
+            int rangeId = AInt32SerializerDeserializer.getInt(bytes1, offset1 + 1);
+            int k = AInt32SerializerDeserializer.getInt(bytes2, offset2 + 1);
+
+            if (rangeId != rangeIdCache) {
+                // Only load new values if the range changed.
+                RangeForwardTaskState rangeState = RangeForwardTaskState.getRangeState(rangeId, ctx);
+                IRangeMap rangeMap = rangeState.getRangeMap();
+                partitionStart = LongPointable.getLong(rangeMap.getMinByteArray(0), rangeMap.getMinStartOffset(0) + 1);
+                long partitionEnd = LongPointable.getLong(rangeMap.getMaxByteArray(0),
+                        rangeMap.getMaxStartOffset(0) + 1);
+                partitionDuration = IntervalPartitionUtil.getPartitionDuration(partitionStart, partitionEnd, k);
+                rangeIdCache = rangeId;
+            }
+
+            int partition = IntervalPartitionUtil.getIntervalPartition(point, partitionStart, partitionDuration, k);
+            aInt.setValue(partition);
+            intSerde.serialize(aInt, out);
+        } catch (HyracksDataException hex) {
+            throw new AlgebricksException(hex);
+        }
+        result.set(resultStorage);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinStartDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinStartDescriptor.java
new file mode 100644
index 0000000..cc1d2f1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionJoinStartDescriptor.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class IntervalPartitionJoinStartDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new IntervalPartitionJoinStartDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args)
+            throws AlgebricksException {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws AlgebricksException {
+                return new IntervalPartitionJoinFunction(args, ctx, true);
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.INTERVAL_PARTITION_JOIN_START;
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionLogic.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionLogic.java
new file mode 100644
index 0000000..5e80fa1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/IntervalPartitionLogic.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.temporal;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+
+/**
+ * Special interval partition logic for interval partition joins.
+ * Interval Partition Logic is used by the interval partition join to determine
+ * which partitions have intervals that meet the given Allen's relation.
+ *
+ * @author prestonc
+ * @see IntervalLogic
+ */
+public class IntervalPartitionLogic {
+
+    /**
+     * Anything from interval 1 is less than anything from interval 2.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     * @throws AsterixException
+     * @see #after(int, int, int, int)
+     */
+    public static boolean before(int s1, int e1, int s2, int e2) {
+        return e1 <= s2;
+    }
+
+    public static boolean after(int s1, int e1, int s2, int e2) {
+        return before(s2, e2, s1, e1);
+    }
+
+    /**
+     * The end of interval 1 is the same as the start of interval 2.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     * @see #metBy(int, int, int, int)
+     */
+    public static boolean meets(int s1, int e1, int s2, int e2) {
+        return e1 == s2;
+    }
+
+    public static boolean metBy(int s1, int e1, int s2, int e2) {
+        return meets(s2, e2, s1, e1);
+    }
+
+    /**
+     * Something at the end of interval 1 is contained as the beginning of interval 2.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     * @see #overlappedBy(int, int, int, int)
+     */
+    public static boolean overlaps(int s1, int e1, int s2, int e2) {
+        return s1 <= s2 && e1 >= s2 && e1 <= e2;
+    }
+
+    public static boolean overlappedBy(int s1, int e1, int s2, int e2) {
+        return overlaps(s2, e2, s1, e1);
+    }
+
+    /**
+     * Something is shared by both interval 1 and interval 2.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     */
+    public static boolean overlapping(int s1, int e1, int s2, int e2) {
+        return s1 <= e2 && e1 >= s2;
+    }
+
+    /**
+     * Anything from interval 1 is contained in the beginning of interval 2.
+     *
+     * @param ip1
+     * @param ip2
+     * @return boolean
+     * @throws AsterixException
+     * @see #startedBy(int, int, int, int)
+     */
+    public static boolean starts(int s1, int e1, int s2, int e2) {
+        return s1 == s2 && e1 <= e2;
+    }
+
+    public static boolean startedBy(int s1, int e1, int s2, int e2) {
+        return starts(s2, e2, s1, e1);
+    }
+
+    /**
+     * Anything from interval 2 is in interval 1.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     * @see #coveredBy(int, int, int, int)
+     */
+    public static boolean covers(int s1, int e1, int s2, int e2) {
+        return s1 <= s2 && e1 >= e2;
+    }
+
+    public static boolean coveredBy(int s1, int e1, int s2, int e2) {
+        return covers(s2, e2, s1, e1);
+    }
+
+    /**
+     * Anything from interval 1 is from the ending part of interval 2.
+     *
+     * @param s1
+     *            First interval start partition
+     * @param e1
+     *            First interval end partition
+     * @param s2
+     *            Second interval start partition
+     * @param e2
+     *            Second interval end partition
+     * @return boolean
+     * @see #endedBy(int, int, int, int)
+     */
+    public static boolean ends(int s1, int e1, int s2, int e2) {
+        return s1 >= s2 && e1 == e2;
+    }
+
+    public static boolean endedBy(int s1, int e1, int s2, int e2) {
+        return ends(s2, e2, s1, e1);
+    }
+
+    public static boolean equals(int s1, int e1, int s2, int e2) {
+        return s1 == s2 && e1 == e2;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinChecker.java
new file mode 100644
index 0000000..4343386
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinChecker.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public abstract class AbstractIntervalInverseMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+
+    private static final long serialVersionUID = 1L;
+
+    public AbstractIntervalInverseMergeJoinChecker(int idLeft, int idRight) {
+        super(idLeft, idRight);
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getEnd(endRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, endRight) <= 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return !checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..f8f94a1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalInverseMergeJoinCheckerFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public abstract class AbstractIntervalInverseMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.SPLIT;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..ec8ecda
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.comparisons.ComparisonHelper;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalLogic;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+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;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public abstract class AbstractIntervalMergeJoinChecker implements IIntervalMergeJoinChecker {
+
+    private static final long serialVersionUID = 1L;
+
+    protected final int idLeft;
+    protected final int idRight;
+
+    protected final IntervalLogic il = new IntervalLogic();
+
+    protected final TaggedValuePointable tvp = (TaggedValuePointable) TaggedValuePointable.FACTORY.createPointable();
+    protected final AIntervalPointable ipLeft = (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
+    protected final AIntervalPointable ipRight = (AIntervalPointable) AIntervalPointable.FACTORY.createPointable();
+
+    protected final ComparisonHelper ch = new ComparisonHelper();
+    protected final IPointable startLeft = VoidPointable.FACTORY.createPointable();
+    protected final IPointable endLeft = VoidPointable.FACTORY.createPointable();
+    protected final IPointable startRight = VoidPointable.FACTORY.createPointable();
+    protected final IPointable endRight = VoidPointable.FACTORY.createPointable();
+
+    public AbstractIntervalMergeJoinChecker(int idLeft, int idRight) {
+        this.idLeft = idLeft;
+        this.idRight = idRight;
+    }
+
+    @Override
+    public boolean checkToRemoveLeftActive() {
+        return true;
+    }
+
+    @Override
+    public boolean checkToRemoveRightActive() {
+        return true;
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getEnd(endLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), endLeft, startRight) > 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getEnd(endRight);
+        return !(ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, endRight) < 0);
+    }
+
+    @Override
+    public boolean checkToLoadNextRightTuple(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean checkToSaveInResult(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return checkToSaveInResult(accessorLeft, accessorLeft.getTupleId(), accessorRight, accessorRight.getTupleId(),
+                false);
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getEnd(endRight);
+        return !(ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, endRight) < 0);
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex, boolean reversed) throws HyracksDataException {
+        if (reversed) {
+            IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, tvp, ipRight);
+            IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, tvp, ipLeft);
+        } else {
+            IntervalJoinUtil.getIntervalPointable(accessorLeft, leftTupleIndex, idLeft, tvp, ipLeft);
+            IntervalJoinUtil.getIntervalPointable(accessorRight, rightTupleIndex, idRight, tvp, ipRight);
+        }
+        return compareInterval(ipLeft, ipRight);
+    }
+
+    @Override
+    public abstract boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight)
+            throws HyracksDataException;
+
+    @Override
+    public abstract boolean compareIntervalPartition(int s1, int e1, int s2, int e2);
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..95fc499
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public abstract class AbstractIntervalMergeJoinCheckerFactory implements IIntervalMergeJoinCheckerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.SPLIT;
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public boolean isOrderAsc() {
+        return true;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..162746f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinChecker.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class AfterIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public AfterIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean checkToRemoveRightActive() {
+        return false;
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, startRight) > 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return !checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.after(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.after(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..09b3020
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AfterIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class AfterIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new AfterIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.REPLICATE;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..2053ab1
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinChecker.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class BeforeIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public BeforeIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean checkToRemoveLeftActive() {
+        return false;
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, startRight) < 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return !checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.before(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.before(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..3e913d2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/BeforeIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class BeforeIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new BeforeIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.REPLICATE;
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..93ce21e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CoveredByIntervalMergeJoinChecker extends AbstractIntervalInverseMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public CoveredByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.coveredBy(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.coveredBy(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinCheckerFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinCheckerFactory.java
index 500ac03..a513cbc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoveredByIntervalMergeJoinCheckerFactory.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.asterix.runtime.operators.joins;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class CoveredByIntervalMergeJoinCheckerFactory extends AbstractIntervalInverseMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new CoveredByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..5535419
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class CoversIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public CoversIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.covers(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.covers(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinCheckerFactory.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinCheckerFactory.java
index 500ac03..cc9b37d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/CoversIntervalMergeJoinCheckerFactory.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.asterix.runtime.operators.joins;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class CoversIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new CoversIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..c25500f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class EndedByIntervalMergeJoinChecker extends EndsIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public EndedByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft, keysRight);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.endedBy(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.endedBy(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..c3a681c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndedByIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class EndedByIntervalMergeJoinCheckerFactory extends AbstractIntervalInverseMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new EndedByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public boolean isOrderAsc() {
+        return false;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..b01faad
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinChecker.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;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class EndsIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public EndsIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+            IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+            IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+            ipLeft.getEnd(endLeft);
+            ipRight.getEnd(endRight);
+            return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), endLeft, endRight) <= 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return !checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.ends(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.ends(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..295bd04
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EndsIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class EndsIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new EndsIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public boolean isOrderAsc() {
+        return false;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EqualsIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EqualsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..b88169f
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/EqualsIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class EqualsIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public EqualsIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.same(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.equals(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..8a3d9c5
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinChecker.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinChecker;
+
+public interface IIntervalMergeJoinChecker extends IMergeJoinChecker {
+
+    public boolean checkToRemoveLeftActive();
+
+    public boolean checkToRemoveRightActive();
+
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException;
+
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2);
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..f2e3d80
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+
+public interface IIntervalMergeJoinCheckerFactory extends IMergeJoinCheckerFactory, Serializable {
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx)
+            throws HyracksDataException;
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IntervalJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IntervalJoinUtil.java
new file mode 100644
index 0000000..7c37f08
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/IntervalJoinUtil.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+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;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+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,
+            TaggedValuePointable tvp, AIntervalPointable ip) {
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId);
+        int length = accessor.getFieldLength(tupleId, fieldId);
+        tvp.set(accessor.getBuffer().array(), start, length);
+        tvp.getValue(ip);
+    }
+
+    public static long getIntervalStart(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+        int length = accessor.getTupleLength(tupleId);
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId) + 1;
+        long intervalStart = AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), start);
+        return intervalStart;
+    }
+
+    public static long getIntervalEnd(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId) + 1;
+        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;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..411ae48
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class MeetsIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public MeetsIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.meets(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.meets(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..c970fd2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MeetsIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class MeetsIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new MeetsIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT_END;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..922f4e0
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinChecker.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class MetByIntervalMergeJoinChecker extends AbstractIntervalInverseMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public MetByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getEnd(endLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), endLeft, startRight) >= 0;
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.metBy(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.metBy(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..fad2d88
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/MetByIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class MetByIntervalMergeJoinCheckerFactory extends AbstractIntervalInverseMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new MetByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT_END;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..2691740
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlappedByIntervalMergeJoinChecker extends AbstractIntervalInverseMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public OverlappedByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.overlappedBy(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.overlappedBy(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
index 500ac03..c47381a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.asterix.runtime.operators.joins;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class OverlappedByIntervalMergeJoinCheckerFactory extends AbstractIntervalInverseMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new OverlappedByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..6c85e36
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinChecker.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class OverlappingIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+    private final long partitionStart;
+
+    public OverlappingIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight, long partitionStart) {
+        super(keysLeft[0], keysRight[0]);
+        this.partitionStart = partitionStart;
+    }
+
+    @Override
+    public boolean checkToSaveInResult(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, idLeft);
+        long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, idRight);
+        if (start0 < partitionStart && start1 < partitionStart) {
+            // Both tuples will match in a different partition.
+            return false;
+        }
+        return super.checkToSaveInResult(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex, boolean reversed) throws HyracksDataException {
+        long start0 = IntervalJoinUtil.getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+        long start1 = IntervalJoinUtil.getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        if (start0 < partitionStart && start1 < partitionStart) {
+            // Both tuples will match in a different partition.
+            return false;
+        }
+        return super.checkToSaveInResult(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex, reversed);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.overlapping(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.overlapping(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..880e181
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappingIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+
+public class OverlappingIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+    private final RangeId rangeId;
+
+    public OverlappingIntervalMergeJoinCheckerFactory(RangeId rangeId) {
+        this.rangeId = rangeId;
+    }
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+        int fieldIndex = 0;
+        RangeForwardTaskState rangeState = RangeForwardTaskState.getRangeState(rangeId.getId(), ctx);
+        IRangeMap rangeMap = rangeState.getRangeMap();
+        if (ATypeTag.INT64.serialize() != rangeMap.getTag(0, 0)) {
+            throw new HyracksDataException("Invalid range map type for interval merge join checker.");
+        }
+        int slot = partition - 1;
+        long partitionStart = 0;
+        // All lookups are on typed values.
+        if (partition <= 0) {
+            partitionStart = LongPointable.getLong(rangeMap.getMinByteArray(fieldIndex),
+                    rangeMap.getMinStartOffset(fieldIndex) + 1);
+        } else if (partition <= rangeMap.getSplitCount()) {
+            partitionStart = LongPointable.getLong(rangeMap.getByteArray(fieldIndex, slot),
+                    rangeMap.getStartOffset(fieldIndex, slot) + 1);
+        } else if (partition > rangeMap.getSplitCount()) {
+            partitionStart = LongPointable.getLong(rangeMap.getMaxByteArray(fieldIndex),
+                    rangeMap.getMaxStartOffset(fieldIndex) + 1);
+        }
+        return new OverlappingIntervalMergeJoinChecker(keys0, keys1, partitionStart);
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.SPLIT;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..3f4c22b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlapsIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public OverlapsIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.overlaps(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.overlaps(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
similarity index 61%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
index 500ac03..0cf3ac1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.asterix.runtime.operators.joins;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+public class OverlapsIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new OverlapsIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..dfb72a7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinChecker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class StartedByIntervalMergeJoinChecker extends StartsIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public StartedByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft, keysRight);
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.startedBy(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.startedBy(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..0938fe2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartedByIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class StartedByIntervalMergeJoinCheckerFactory extends AbstractIntervalInverseMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new StartedByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinChecker.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..20526e2
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinChecker.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;
+
+import org.apache.asterix.om.pointables.nonvisitor.AIntervalPointable;
+import org.apache.asterix.runtime.evaluators.functions.temporal.IntervalPartitionLogic;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class StartsIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public StartsIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, startRight) == 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return !checkToSaveInMemory(accessorLeft, accessorRight);
+    }
+
+    @Override
+    public boolean checkToLoadNextRightTuple(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        IntervalJoinUtil.getIntervalPointable(accessorLeft, idLeft, tvp, ipLeft);
+        IntervalJoinUtil.getIntervalPointable(accessorRight, idRight, tvp, ipRight);
+        ipLeft.getStart(startLeft);
+        ipRight.getStart(startRight);
+        return ch.compare(ipLeft.getTypeTag(), ipRight.getTypeTag(), startLeft, startRight) >= 0;
+    }
+
+    @Override
+    public boolean compareInterval(AIntervalPointable ipLeft, AIntervalPointable ipRight) throws HyracksDataException {
+        return il.starts(ipLeft, ipRight);
+    }
+
+    @Override
+    public boolean compareIntervalPartition(int s1, int e1, int s2, int e2) {
+        return IntervalPartitionLogic.starts(s1, e1, s2, e2);
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinCheckerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..924b442
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/StartsIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+
+public class StartsIntervalMergeJoinCheckerFactory extends AbstractIntervalMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IIntervalMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        return new StartsIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/ActiveSweepManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/ActiveSweepManager.java
new file mode 100644
index 0000000..84bd262
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/ActiveSweepManager.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.operators.joins.intervalindex;
+
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IntervalJoinUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.IPartitionedDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class ActiveSweepManager {
+
+    private static final Logger LOGGER = Logger.getLogger(ActiveSweepManager.class.getName());
+
+    private final int partition;
+    private final int key;
+
+    private final IPartitionedDeletableTupleBufferManager bufferManager;
+    private final PriorityQueue<EndPointIndexItem> indexQueue;
+    private EndPointIndexItem item = null;
+    private final LinkedList<TuplePointer> active = new LinkedList<>();
+
+    public ActiveSweepManager(IPartitionedDeletableTupleBufferManager bufferManager, int key, int joinBranch,
+            Comparator<EndPointIndexItem> endPointComparator) {
+        this.bufferManager = bufferManager;
+        this.key = key;
+        this.partition = joinBranch;
+        indexQueue = new PriorityQueue<>(16, endPointComparator);
+    }
+
+    public boolean addTuple(ITupleAccessor accessor, TuplePointer tp) throws HyracksDataException {
+        if (bufferManager.insertTuple(partition, accessor, accessor.getTupleId(), tp)) {
+            EndPointIndexItem e = new EndPointIndexItem(tp, EndPointIndexItem.END_POINT,
+                    IntervalJoinUtil.getIntervalEnd(accessor, accessor.getTupleId(), key));
+            indexQueue.add(e);
+            active.add(tp);
+            item = indexQueue.peek();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Add to memory (partition: " + partition + " index: " + e + ").");
+            }
+            return true;
+        }
+        return false;
+    }
+
+    public void removeTop() throws HyracksDataException {
+        // Remove from active.
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Remove top from memory (partition: " + partition + " index: " + item + ").");
+        }
+        bufferManager.deleteTuple(partition, item.getTuplePointer());
+        active.remove(item.getTuplePointer());
+        indexQueue.remove(item);
+        item = indexQueue.peek();
+    }
+
+    public long getTopPoint() {
+        return item.getPoint();
+    }
+
+    public List<TuplePointer> getActiveList() {
+        return active;
+    }
+
+    public boolean isEmpty() {
+        return indexQueue.isEmpty();
+    }
+
+    public boolean hasRecords() {
+        return !indexQueue.isEmpty();
+    }
+
+    public void clear() throws HyracksDataException {
+        for (TuplePointer leftTp : active) {
+            bufferManager.deleteTuple(partition, leftTp);
+        }
+        indexQueue.clear();
+        active.clear();
+        item = null;
+        bufferManager.clearPartition(partition);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/EndPointIndexItem.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/EndPointIndexItem.java
new file mode 100644
index 0000000..f34449d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/EndPointIndexItem.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.intervalindex;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class EndPointIndexItem implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final byte START_POINT = 0;
+    public static final byte END_POINT = 1;
+
+    public static final Comparator<EndPointIndexItem> EndPointAscComparator = new Comparator<EndPointIndexItem>() {
+        @Override
+        public int compare(EndPointIndexItem epi1, EndPointIndexItem epi2) {
+            int c = (int) (epi1.getPoint() - epi2.getPoint());
+            if (c == 0) {
+                c = epi1.getStart() - epi2.getStart();
+            }
+            return c;
+        }
+
+    };
+
+    public static final Comparator<EndPointIndexItem> EndPointDescComparator = new Comparator<EndPointIndexItem>() {
+        @Override
+        public int compare(EndPointIndexItem epi1, EndPointIndexItem epi2) {
+            int c = (int) (epi2.getPoint() - epi1.getPoint());
+            if (c == 0) {
+                c = epi2.getStart() - epi1.getStart();
+            }
+            return c;
+        }
+
+    };
+
+    private TuplePointer tp;
+    private byte start;
+    private long point;
+
+    public EndPointIndexItem() {
+        reset(new TuplePointer(), (byte) -1, 0);
+    }
+
+    public EndPointIndexItem(TuplePointer tp, byte start, long point) {
+        reset(tp, start, point);
+    }
+
+    public void reset(EndPointIndexItem item) {
+        reset(item.getTuplePointer(), item.getStart(), item.getPoint());
+    }
+
+    public void reset(TuplePointer tp, byte start, long point) {
+        this.tp = tp;
+        this.start = start;
+        this.point = point;
+    }
+
+    public TuplePointer getTuplePointer() {
+        return tp;
+    }
+
+    public byte getStart() {
+        return start;
+    }
+
+    public long getPoint() {
+        return point;
+    }
+
+    @Override
+    public String toString() {
+        return "EndPointIndexItem " + tp + " " + (start == START_POINT ? "start" : "end") + ": " + point;
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IndexJoinTaskState.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IndexJoinTaskState.java
new file mode 100644
index 0000000..16d4ad7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IndexJoinTaskState.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.intervalindex;
+
+import java.util.Comparator;
+
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.std.join.MergeJoinTaskState;
+
+public class IndexJoinTaskState extends MergeJoinTaskState {
+    protected IntervalIndexJoiner indexJoiner;
+    protected Comparator<EndPointIndexItem> endPointComparator;
+    protected byte point;
+
+    public IndexJoinTaskState(JobId jobId, TaskId taskId) {
+        super(jobId, taskId);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoinOperatorDescriptor.java
new file mode 100644
index 0000000..fbddfef
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoinOperatorDescriptor.java
@@ -0,0 +1,294 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.intervalindex;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+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.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.join.MergeBranchStatus.Stage;
+import org.apache.hyracks.dataflow.std.join.MergeJoinLocks;
+
+public class IntervalIndexJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int LEFT_ACTIVITY_ID = 0;
+    private static final int RIGHT_ACTIVITY_ID = 1;
+    private final int[] leftKeys;
+    private final int[] rightKeys;
+    private final int memoryForJoin;
+    private final IIntervalMergeJoinCheckerFactory imjcf;
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalIndexJoinOperatorDescriptor.class.getName());
+
+    public IntervalIndexJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin, int[] leftKeys,
+            int[] rightKeys, RecordDescriptor recordDescriptor, IIntervalMergeJoinCheckerFactory imjcf) {
+        super(spec, 2, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.memoryForJoin = memoryForJoin;
+        this.imjcf = imjcf;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        MergeJoinLocks locks = new MergeJoinLocks();
+
+        ActivityId leftAid = new ActivityId(odId, LEFT_ACTIVITY_ID);
+        ActivityId rightAid = new ActivityId(odId, RIGHT_ACTIVITY_ID);
+
+        IActivity leftAN = new LeftJoinerActivityNode(leftAid, rightAid, locks);
+        IActivity rightAN = new RightDataActivityNode(rightAid, leftAid, locks);
+
+        builder.addActivity(this, rightAN);
+        builder.addSourceEdge(1, rightAN, 0);
+
+        builder.addActivity(this, leftAN);
+        builder.addSourceEdge(0, leftAN, 0);
+        builder.addTargetEdge(0, leftAN, 0);
+    }
+
+    private class LeftJoinerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final MergeJoinLocks locks;
+
+        public LeftJoinerActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new LeftJoinerOperator(ctx, partition, inRecordDesc);
+        }
+
+        private class LeftJoinerOperator extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+            private final IHyracksTaskContext ctx;
+            private final int partition;
+            private final RecordDescriptor leftRd;
+            private IndexJoinTaskState state;
+            private boolean first = true;
+
+            public LeftJoinerOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.leftRd = inRecordDesc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    writer.open();
+                    state = new IndexJoinTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));;
+                    state.leftRd = leftRd;
+                    state.point = imjcf.isOrderAsc() ? EndPointIndexItem.START_POINT : EndPointIndexItem.END_POINT;
+                    state.endPointComparator = imjcf.isOrderAsc() ? EndPointIndexItem.EndPointAscComparator
+                            : EndPointIndexItem.EndPointDescComparator;
+                    ctx.setStateObject(state);
+                    locks.getRight(partition).signal();
+
+                    do {
+                        // Continue after joiner created in right branch.
+                        if (state.indexJoiner == null) {
+                            locks.getLeft(partition).await();
+                        }
+                    } while (state.indexJoiner == null);
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageOpen();
+                    locks.getRight(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+                if (first) {
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    state.indexJoiner.setFrame(LEFT_ACTIVITY_ID, buffer);
+                    state.indexJoiner.processLeftFrame(writer);
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[LEFT_ACTIVITY_ID].noMore();
+                    if (state.failed) {
+                        writer.fail();
+                    } else {
+                        state.indexJoiner.processLeftClose(writer);
+                        writer.close();
+                    }
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageClose();
+                    locks.getRight(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+        }
+    }
+
+    private class RightDataActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId joinAid;
+        private final MergeJoinLocks locks;
+
+        public RightDataActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.joinAid = joinAid;
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new RightDataOperator(ctx, partition, inRecordDesc);
+        }
+
+        private class RightDataOperator extends AbstractUnaryInputSinkOperatorNodePushable {
+
+            private int partition;
+            private IHyracksTaskContext ctx;
+            private final RecordDescriptor rightRd;
+            private IndexJoinTaskState state;
+            private boolean first = true;
+
+            public RightDataOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.rightRd = inRecordDesc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    do {
+                        // Wait for the state to be set in the context form Left.
+                        state = (IndexJoinTaskState) ctx.getStateObject(new TaskId(joinAid, partition));
+                        if (state == null) {
+                            locks.getRight(partition).await();
+                        }
+                    } while (state == null);
+                    state.rightRd = rightRd;
+                    state.indexJoiner = new IntervalIndexJoiner(ctx, memoryForJoin, partition, state.status, locks,
+                            state.endPointComparator, imjcf, leftKeys, rightKeys, state.leftRd, state.rightRd);
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageOpen();
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+                if (first) {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    while (!state.status.continueRightLoad
+                            && state.status.branch[LEFT_ACTIVITY_ID].getStatus() != Stage.CLOSED) {
+                        // Wait for the state to request right frame unless left has finished.
+                        locks.getRight(partition).await();
+                    }
+                    state.indexJoiner.setFrame(RIGHT_ACTIVITY_ID, buffer);
+                    state.status.continueRightLoad = false;
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageClose();
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoiner.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoiner.java
new file mode 100644
index 0000000..d3303b6
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/IntervalIndexJoiner.java
@@ -0,0 +1,499 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.intervalindex;
+
+import java.util.Comparator;
+import java.util.List;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinChecker;
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.buffermanager.IPartitionedDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.ITuplePointerAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.TupleAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.VPartitionDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.join.AbstractMergeJoiner;
+import org.apache.hyracks.dataflow.std.join.MergeJoinLocks;
+import org.apache.hyracks.dataflow.std.join.MergeStatus;
+import org.apache.hyracks.dataflow.std.join.RunFileStream;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+/**
+ * Interval Index 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 both right and left use memory to maintain active intervals for the join.
+ */
+public class IntervalIndexJoiner extends AbstractMergeJoiner {
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalIndexJoiner.class.getName());
+
+    private final IPartitionedDeletableTupleBufferManager bufferManager;
+
+    private final ActiveSweepManager[] activeManager;
+    private final ITuplePointerAccessor[] memoryAccessor;
+    private final int[] streamIndex;
+    private final RunFileStream[] runFileStream;
+
+    //    private final LinkedList<TuplePointer> buffer = new LinkedList<>();
+
+    private final IIntervalMergeJoinChecker imjc;
+
+    private final byte point;
+
+    private final int leftKey;
+    private final int rightKey;
+
+    private long joinComparisonCount = 0;
+    private long joinResultCount = 0;
+    private long leftSpillCount = 0;
+    private long rightSpillCount = 0;
+    private long[] spillFileCount = { 0, 0 };
+    private long[] spillReadCount = { 0, 0 };
+    private long[] spillWriteCount = { 0, 0 };
+
+    public IntervalIndexJoiner(IHyracksTaskContext ctx, int memorySize, int partition, MergeStatus status,
+            MergeJoinLocks locks, Comparator<EndPointIndexItem> endPointComparator,
+            IIntervalMergeJoinCheckerFactory imjcf, int[] leftKeys, int[] rightKeys, RecordDescriptor leftRd,
+            RecordDescriptor rightRd) throws HyracksDataException {
+        super(ctx, partition, status, locks, leftRd, rightRd);
+        this.point = imjcf.isOrderAsc() ? EndPointIndexItem.START_POINT : EndPointIndexItem.END_POINT;
+
+        this.imjc = imjcf.createMergeJoinChecker(leftKeys, rightKeys, partition, ctx);
+
+        this.leftKey = leftKeys[0];
+        this.rightKey = rightKeys[0];
+
+        RecordDescriptor[] recordDescriptors = new RecordDescriptor[JOIN_PARTITIONS];
+        recordDescriptors[LEFT_PARTITION] = leftRd;
+        recordDescriptors[RIGHT_PARTITION] = rightRd;
+
+        streamIndex = new int[JOIN_PARTITIONS];
+        streamIndex[LEFT_PARTITION] = TupleAccessor.UNSET;
+        streamIndex[RIGHT_PARTITION] = TupleAccessor.UNSET;
+
+        if (memorySize < 5) {
+            throw new HyracksDataException(
+                    "IntervalIndexJoiner does not have enough memory (needs > 4, got " + memorySize + ").");
+        }
+        //        bufferManager = new VPartitionDeletableTupleBufferManager(ctx,
+        //                VPartitionDeletableTupleBufferManager.NO_CONSTRAIN, JOIN_PARTITIONS,
+        //                (memorySize - 4) * ctx.getInitialFrameSize(), recordDescriptors);
+        bufferManager = new VPartitionDeletableTupleBufferManager(ctx,
+                VPartitionDeletableTupleBufferManager.NO_CONSTRAIN, JOIN_PARTITIONS,
+                memorySize * ctx.getInitialFrameSize(), recordDescriptors);
+        memoryAccessor = new ITuplePointerAccessor[JOIN_PARTITIONS];
+        memoryAccessor[LEFT_PARTITION] = bufferManager.getTuplePointerAccessor(leftRd);
+        memoryAccessor[RIGHT_PARTITION] = bufferManager.getTuplePointerAccessor(rightRd);
+
+        activeManager = new ActiveSweepManager[JOIN_PARTITIONS];
+        activeManager[LEFT_PARTITION] = new ActiveSweepManager(bufferManager, leftKey, LEFT_PARTITION,
+                endPointComparator);
+        activeManager[RIGHT_PARTITION] = new ActiveSweepManager(bufferManager, rightKey, RIGHT_PARTITION,
+                endPointComparator);
+
+        // Run files for both branches
+        runFileStream = new RunFileStream[JOIN_PARTITIONS];
+        runFileStream[LEFT_PARTITION] = new RunFileStream(ctx, "left", status.branch[LEFT_PARTITION]);
+        runFileStream[RIGHT_PARTITION] = new RunFileStream(ctx, "right", status.branch[RIGHT_PARTITION]);
+
+        LOGGER.setLevel(Level.FINE);
+        System.out.println("IntervalIndexJoiner: Logging level is: " + LOGGER.getLevel());
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("IntervalIndexJoiner has started partition " + partition + " with " + memorySize
+                    + " frames of memory.");
+        }
+    }
+
+    private void addToResult(IFrameTupleAccessor accessor1, int index1, IFrameTupleAccessor accessor2, int index2,
+            boolean reversed, IFrameWriter writer) throws HyracksDataException {
+        if (reversed) {
+            FrameUtils.appendConcatToWriter(writer, resultAppender, accessor2, index2, accessor1, index1);
+        } else {
+            FrameUtils.appendConcatToWriter(writer, resultAppender, accessor1, index1, accessor2, index2);
+        }
+        joinResultCount++;
+    }
+
+    private void flushMemory(int partition) throws HyracksDataException {
+        activeManager[partition].clear();
+    }
+
+    private TupleStatus loadSpilledTuple(int partition) throws HyracksDataException {
+        if (!inputAccessor[partition].exists()) {
+            if (!runFileStream[partition].loadNextBuffer(inputAccessor[partition])) {
+                return TupleStatus.EMPTY;
+            }
+        }
+        return TupleStatus.LOADED;
+    }
+
+    private TupleStatus loadTuple(int partition) throws HyracksDataException {
+        TupleStatus loaded;
+        if (status.branch[partition].isRunFileReading()) {
+            loaded = loadSpilledTuple(partition);
+            if (loaded.isEmpty()) {
+                continueStream(partition, inputAccessor[partition]);
+                loaded = loadTuple(partition);
+            }
+        } else {
+            loaded = loadMemoryTuple(partition);
+        }
+        return loaded;
+    }
+
+    /**
+     * Ensures a frame exists for the right branch, either from memory or the run file.
+     *
+     * @throws HyracksDataException
+     */
+    private TupleStatus loadRightTuple() throws HyracksDataException {
+        TupleStatus loaded = loadTuple(RIGHT_PARTITION);
+        if (loaded == TupleStatus.UNKNOWN) {
+            loaded = pauseAndLoadRightTuple();
+        }
+        return loaded;
+    }
+
+    /**
+     * Ensures a frame exists for the left branch, either from memory or the run file.
+     *
+     * @throws HyracksDataException
+     */
+    private TupleStatus loadLeftTuple() throws HyracksDataException {
+        return loadTuple(LEFT_PARTITION);
+    }
+
+    @Override
+    public void processLeftFrame(IFrameWriter writer) throws HyracksDataException {
+        TupleStatus leftTs = loadLeftTuple();
+        TupleStatus rightTs = loadRightTuple();
+        while (leftTs.isKnown() && checkHasMoreProcessing(leftTs, LEFT_PARTITION, RIGHT_PARTITION)
+                && checkHasMoreProcessing(rightTs, RIGHT_PARTITION, LEFT_PARTITION)) {
+            if (status.branch[RIGHT_PARTITION].isRunFileWriting()) {
+                // Right side from disk
+                rightTs = processRightTupleSpill(writer);
+            } else if (status.branch[LEFT_PARTITION].isRunFileWriting()) {
+                // Left side from disk
+                leftTs = processLeftTupleSpill(writer);
+            } else {
+                if (leftTs.isEmpty() || (rightTs.isLoaded() && checkToProcessRightTuple())) {
+                    // Right side from stream
+                    processRightTuple(writer);
+                    rightTs = loadRightTuple();
+                } else {
+                    // Left side from stream
+                    processLeftTuple(writer);
+                    leftTs = loadLeftTuple();
+                }
+            }
+        }
+    }
+
+    @Override
+    public void processLeftClose(IFrameWriter writer) throws HyracksDataException {
+        processLeftFrame(writer);
+
+        resultAppender.write(writer, true);
+        activeManager[LEFT_PARTITION].clear();
+        activeManager[RIGHT_PARTITION].clear();
+        runFileStream[LEFT_PARTITION].close();
+        runFileStream[RIGHT_PARTITION].close();
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("IntervalIndexJoiner statitics: " + joinComparisonCount + " comparisons, " + joinResultCount
+                    + " results, left[" + leftSpillCount + " spills, " + runFileStream[LEFT_PARTITION].getFileCount()
+                    + " files, " + runFileStream[LEFT_PARTITION].getWriteCount() + " written, "
+                    + runFileStream[LEFT_PARTITION].getReadCount() + " read]. right[" + rightSpillCount + " spills, "
+                    + runFileStream[RIGHT_PARTITION].getFileCount() + " files, "
+                    + runFileStream[RIGHT_PARTITION].getWriteCount() + " written, "
+                    + runFileStream[RIGHT_PARTITION].getReadCount() + " read].");
+        }
+
+    }
+
+    private boolean checkHasMoreProcessing(TupleStatus ts, int partition, int joinPartition) {
+        return ts.isLoaded() || status.branch[partition].isRunFileWriting()
+                || (checkHasMoreTuples(joinPartition) && activeManager[partition].hasRecords());
+    }
+
+    private boolean checkHasMoreTuples(int partition) {
+        return status.branch[partition].hasMore() || status.branch[partition].isRunFileReading();
+    }
+
+    private boolean checkToProcessRightTuple() {
+        long leftStart = IntervalJoinUtil.getIntervalStart(inputAccessor[LEFT_PARTITION], leftKey);
+        long rightStart = IntervalJoinUtil.getIntervalStart(inputAccessor[RIGHT_PARTITION], rightKey);
+        if (leftStart < rightStart) {
+            // Left stream has next tuple, check if right active must be updated first.
+            return activeManager[RIGHT_PARTITION].hasRecords()
+                    && activeManager[RIGHT_PARTITION].getTopPoint() < leftStart;
+        } else {
+            // Right stream has next tuple, check if left active must be update first.
+            return !(activeManager[LEFT_PARTITION].hasRecords()
+                    && activeManager[LEFT_PARTITION].getTopPoint() < rightStart);
+        }
+    }
+
+    private boolean checkToProcessAdd(long startMemory, long endMemory) {
+        return startMemory < endMemory;
+    }
+
+    private TupleStatus processLeftTupleSpill(IFrameWriter writer) throws HyracksDataException {
+        // Process left tuples one by one, check them with active memory from the right branch.
+        int count = 0;
+        TupleStatus ts = loadLeftTuple();
+        while (ts.isLoaded() && activeManager[RIGHT_PARTITION].hasRecords()) {
+            long sweep = activeManager[RIGHT_PARTITION].getTopPoint();
+            if (checkToProcessAdd(IntervalJoinUtil.getIntervalStart(inputAccessor[LEFT_PARTITION], leftKey), sweep)
+                    || !imjc.checkToRemoveRightActive()) {
+                // Add individual tuples.
+                processTupleJoin(activeManager[RIGHT_PARTITION].getActiveList(), memoryAccessor[RIGHT_PARTITION],
+                        inputAccessor[LEFT_PARTITION], true, writer);
+                runFileStream[LEFT_PARTITION].addToRunFile(inputAccessor[LEFT_PARTITION]);
+                inputAccessor[LEFT_PARTITION].next();
+                ts = loadLeftTuple();
+                ++count;
+            } else {
+                // Remove from active.
+                activeManager[RIGHT_PARTITION].removeTop();
+            }
+        }
+
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Spill for " + count + " left tuples");
+        }
+
+        // Memory is empty and we can start processing the run file.
+        if (activeManager[RIGHT_PARTITION].isEmpty() || ts.isEmpty()) {
+            unfreezeAndContinue(LEFT_PARTITION, inputAccessor[LEFT_PARTITION]);
+            ts = loadLeftTuple();
+        }
+        return ts;
+    }
+
+    private TupleStatus processRightTupleSpill(IFrameWriter writer) throws HyracksDataException {
+        // Process left tuples one by one, check them with active memory from the right branch.
+        int count = 0;
+        TupleStatus ts = loadRightTuple();
+        while (ts.isLoaded() && activeManager[LEFT_PARTITION].hasRecords() && inputAccessor[RIGHT_PARTITION].exists()) {
+            long sweep = activeManager[LEFT_PARTITION].getTopPoint();
+            if (checkToProcessAdd(IntervalJoinUtil.getIntervalStart(inputAccessor[RIGHT_PARTITION], rightKey), sweep)
+                    || !imjc.checkToRemoveLeftActive()) {
+                // Add individual tuples.
+                processTupleJoin(activeManager[LEFT_PARTITION].getActiveList(), memoryAccessor[LEFT_PARTITION],
+                        inputAccessor[RIGHT_PARTITION], false, writer);
+                runFileStream[RIGHT_PARTITION].addToRunFile(inputAccessor[RIGHT_PARTITION]);
+                inputAccessor[RIGHT_PARTITION].next();
+                ts = loadRightTuple();
+                ++count;
+            } else {
+                // Remove from active.
+                activeManager[LEFT_PARTITION].removeTop();
+            }
+        }
+
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Spill for " + count + " right tuples");
+        }
+
+        // Memory is empty and we can start processing the run file.
+        if (!activeManager[LEFT_PARTITION].hasRecords() || ts.isEmpty()) {
+            unfreezeAndContinue(RIGHT_PARTITION, inputAccessor[RIGHT_PARTITION]);
+            ts = loadRightTuple();
+        }
+        return ts;
+    }
+
+    private void processLeftTuple(IFrameWriter writer) throws HyracksDataException {
+        // Process endpoints
+        do {
+            if ((!activeManager[LEFT_PARTITION].hasRecords()
+                    || checkToProcessAdd(IntervalJoinUtil.getIntervalStart(inputAccessor[LEFT_PARTITION], leftKey),
+                            activeManager[LEFT_PARTITION].getTopPoint()))
+                    || !imjc.checkToRemoveLeftActive()) {
+                // Add to active, end point index and buffer.
+                TuplePointer tp = new TuplePointer();
+                if (activeManager[LEFT_PARTITION].addTuple(inputAccessor[LEFT_PARTITION], tp)) {
+                    processTupleJoin(activeManager[RIGHT_PARTITION].getActiveList(), memoryAccessor[RIGHT_PARTITION],
+                            inputAccessor[LEFT_PARTITION], true, writer);
+                    //                    buffer.add(tp);
+                } else {
+                    // Spill case
+                    freezeAndSpill();
+                    break;
+                }
+                inputAccessor[LEFT_PARTITION].next();
+            } else {
+                // Remove from active.
+                activeManager[LEFT_PARTITION].removeTop();
+            }
+        } while (loadLeftTuple().isLoaded() && loadRightTuple().isLoaded() && !checkToProcessRightTuple());
+
+        // Add Results
+        //        if (!buffer.isEmpty()) {
+        //            processActiveJoin(activeManager[RIGHT_PARTITION].getActiveList(), memoryAccessor[RIGHT_PARTITION], buffer,
+        //                    memoryAccessor[LEFT_PARTITION], true, writer);
+        //        }
+    }
+
+    private void processRightTuple(IFrameWriter writer) throws HyracksDataException {
+        // Process endpoints
+        do {
+            if ((!activeManager[RIGHT_PARTITION].hasRecords()
+                    || checkToProcessAdd(IntervalJoinUtil.getIntervalStart(inputAccessor[RIGHT_PARTITION], rightKey),
+                            activeManager[RIGHT_PARTITION].getTopPoint()))
+                    || !imjc.checkToRemoveRightActive()) {
+                // Add to active, end point index and buffer.
+                TuplePointer tp = new TuplePointer();
+                if (activeManager[RIGHT_PARTITION].addTuple(inputAccessor[RIGHT_PARTITION], tp)) {
+                    processTupleJoin(activeManager[LEFT_PARTITION].getActiveList(), memoryAccessor[LEFT_PARTITION],
+                            inputAccessor[RIGHT_PARTITION], false, writer);
+                    //                    buffer.add(tp);
+                } else {
+                    // Spill case
+                    freezeAndSpill();
+                    break;
+                }
+                inputAccessor[RIGHT_PARTITION].next();
+            } else {
+                // Remove from active.
+                activeManager[RIGHT_PARTITION].removeTop();
+            }
+        } while (loadRightTuple().isLoaded() && checkToProcessRightTuple());
+
+        // Add Results
+        //        if (!buffer.isEmpty()) {
+        //            processActiveJoin(activeManager[LEFT_PARTITION].getActiveList(), memoryAccessor[LEFT_PARTITION], buffer,
+        //                    memoryAccessor[RIGHT_PARTITION], false, writer);
+        //        }
+    }
+
+    //    private void processActiveJoin(List<TuplePointer> outer, ITuplePointerAccessor outerAccessor,
+    //            List<TuplePointer> inner, ITuplePointerAccessor innerAccessor, boolean reversed, IFrameWriter writer)
+    //            throws HyracksDataException {
+    //        for (TuplePointer outerTp : outer) {
+    //            outerAccessor.reset(outerTp);
+    //            for (TuplePointer innerTp : inner) {
+    //                innerAccessor.reset(innerTp);
+    //                if (imjc.checkToSaveInResult(outerAccessor, outerTp.getTupleIndex(), innerAccessor,
+    //                        innerTp.getTupleIndex(), reversed)) {
+    //                    addToResult(outerAccessor, outerTp.getTupleIndex(), innerAccessor, innerTp.getTupleIndex(),
+    //                            reversed, writer);
+    //                }
+    //                joinComparisonCount++;
+    //            }
+    //        }
+    //        if (LOGGER.isLoggable(Level.FINE)) {
+    //            LOGGER.fine("Sweep for " + buffer.size() + " tuples");
+    //        }
+    //        buffer.clear();
+    //    }
+
+    private void processTupleJoin(List<TuplePointer> outer, ITuplePointerAccessor outerAccessor,
+            ITupleAccessor tupleAccessor, boolean reversed, IFrameWriter writer) throws HyracksDataException {
+        for (TuplePointer outerTp : outer) {
+            outerAccessor.reset(outerTp);
+            if (imjc.checkToSaveInResult(outerAccessor, outerTp.getTupleIndex(), tupleAccessor,
+                    tupleAccessor.getTupleId(), reversed)) {
+                addToResult(outerAccessor, outerTp.getTupleIndex(), tupleAccessor, tupleAccessor.getTupleId(), reversed,
+                        writer);
+            }
+            joinComparisonCount++;
+        }
+    }
+
+    private void freezeAndSpill() throws HyracksDataException {
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("freeze snapshot: " + frameCounts[RIGHT_PARTITION] + " right, " + frameCounts[LEFT_PARTITION]
+                    + " left, left[" + bufferManager.getNumTuples(LEFT_PARTITION) + " memory]. right["
+                    + bufferManager.getNumTuples(RIGHT_PARTITION) + " memory].");
+        }
+        if (bufferManager.getNumTuples(LEFT_PARTITION) > bufferManager.getNumTuples(RIGHT_PARTITION)) {
+            runFileStream[RIGHT_PARTITION].startRunFileWriting();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Memory is full. Freezing the left branch. (Left memory tuples: "
+                        + bufferManager.getNumTuples(LEFT_PARTITION) + ", Right memory tuples: "
+                        + bufferManager.getNumTuples(RIGHT_PARTITION) + ")");
+            }
+            bufferManager.printStats("memory details");
+            rightSpillCount++;
+        } else {
+            runFileStream[LEFT_PARTITION].startRunFileWriting();
+            if (LOGGER.isLoggable(Level.FINE)) {
+                LOGGER.fine("Memory is full. Freezing the right branch. (Left memory tuples: "
+                        + bufferManager.getNumTuples(LEFT_PARTITION) + ", Right memory tuples: "
+                        + bufferManager.getNumTuples(RIGHT_PARTITION) + ")");
+            }
+            bufferManager.printStats("memory details");
+            leftSpillCount++;
+        }
+    }
+
+    private void continueStream(int diskPartition, ITupleAccessor accessor) throws HyracksDataException {
+        runFileStream[diskPartition].closeRunFileReading();
+        accessor.reset(inputBuffer[diskPartition]);
+        accessor.setTupleId(streamIndex[diskPartition]);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Continue with stream (" + diskPartition + ").");
+        }
+    }
+
+    private void unfreezeAndContinue(int frozenPartition, ITupleAccessor accessor) throws HyracksDataException {
+        int flushPartition = frozenPartition == LEFT_PARTITION ? RIGHT_PARTITION : LEFT_PARTITION;
+        runFileStream[frozenPartition].flushAndStopRunFile(accessor);
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("snapshot(" + frozenPartition + "): " + frameCounts[RIGHT_PARTITION] + " right, "
+                    + frameCounts[LEFT_PARTITION] + " left, left[" + bufferManager.getNumTuples(LEFT_PARTITION)
+                    + " memory, " + leftSpillCount + " spills, "
+                    + (runFileStream[LEFT_PARTITION].getFileCount() - spillFileCount[LEFT_PARTITION]) + " files, "
+                    + (runFileStream[LEFT_PARTITION].getWriteCount() - spillWriteCount[LEFT_PARTITION]) + " written, "
+                    + (runFileStream[LEFT_PARTITION].getReadCount() - spillReadCount[LEFT_PARTITION]) + " read]. right["
+                    + bufferManager.getNumTuples(RIGHT_PARTITION) + " memory, " + +rightSpillCount + " spills, "
+                    + (runFileStream[RIGHT_PARTITION].getFileCount() - spillFileCount[RIGHT_PARTITION]) + " files, "
+                    + (runFileStream[RIGHT_PARTITION].getWriteCount() - spillWriteCount[RIGHT_PARTITION]) + " written, "
+                    + (runFileStream[RIGHT_PARTITION].getReadCount() - spillReadCount[RIGHT_PARTITION]) + " read].");
+            spillFileCount[LEFT_PARTITION] = runFileStream[LEFT_PARTITION].getFileCount();
+            spillReadCount[LEFT_PARTITION] = runFileStream[LEFT_PARTITION].getReadCount();
+            spillWriteCount[LEFT_PARTITION] = runFileStream[LEFT_PARTITION].getWriteCount();
+            spillFileCount[RIGHT_PARTITION] = runFileStream[RIGHT_PARTITION].getFileCount();
+            spillReadCount[RIGHT_PARTITION] = runFileStream[RIGHT_PARTITION].getReadCount();
+            spillWriteCount[RIGHT_PARTITION] = runFileStream[RIGHT_PARTITION].getWriteCount();
+        }
+        flushMemory(flushPartition);
+        if ((LEFT_PARTITION == frozenPartition && !status.branch[LEFT_PARTITION].isRunFileReading())
+                || (RIGHT_PARTITION == frozenPartition && !status.branch[RIGHT_PARTITION].isRunFileReading())) {
+            streamIndex[frozenPartition] = accessor.getTupleId();
+        }
+        runFileStream[frozenPartition].startReadingRunFile(accessor);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Unfreezing (" + frozenPartition + ").");
+        }
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/TuplePrinterUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/TuplePrinterUtil.java
new file mode 100644
index 0000000..914f3f4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalindex/TuplePrinterUtil.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.intervalindex;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.adm.AObjectPrinterFactory;
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class TuplePrinterUtil {
+
+    private static final IPrinter printer = AObjectPrinterFactory.INSTANCE.createPrinter();
+
+    private TuplePrinterUtil() {
+    }
+
+    public static void printTuple(String message, ITupleAccessor accessor) throws HyracksDataException {
+        if (accessor.exists()) {
+            printTuple(message, accessor, accessor.getTupleId());
+        } else {
+            System.err.print(String.format("%1$-" + 15 + "s", message) + " --");
+            System.err.print("no tuple");
+            System.err.println();
+        }
+    }
+
+    public static void printTuple(String message, IFrameTupleAccessor accessor, int tupleId)
+            throws HyracksDataException {
+        System.err.print(String.format("%1$-" + 15 + "s", message) + " --");
+        int fields = accessor.getFieldCount();
+        for (int i = 0; i < fields; ++i) {
+            System.err.print(" " + i + ": ");
+            int fieldStartOffset = accessor.getFieldStartOffset(tupleId, i);
+            int fieldSlotsLength = accessor.getFieldSlotsLength();
+            int tupleStartOffset = accessor.getTupleStartOffset(tupleId);
+            printer.print(accessor.getBuffer().array(), fieldStartOffset + fieldSlotsLength + tupleStartOffset,
+                    accessor.getFieldLength(tupleId, i), System.err);
+        }
+        System.err.println();
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionComputerFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionComputerFactory.java
new file mode 100644
index 0000000..38ab073
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionComputerFactory.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.intervalpartition;
+
+import org.apache.asterix.runtime.operators.joins.IntervalJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntervalPartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final int intervalFieldId;
+    private final int k;
+    private final long partitionStart;
+    private final long partitionDuration;
+
+    public IntervalPartitionComputerFactory(int intervalFieldId, int k, long partitionStart, long partitionEnd)
+            throws HyracksDataException {
+        this.intervalFieldId = intervalFieldId;
+        this.k = k;
+        this.partitionStart = partitionStart;
+        this.partitionDuration = IntervalPartitionUtil.getPartitionDuration(partitionStart, partitionEnd, k);
+    }
+
+    @Override
+    public ITuplePartitionComputer createPartitioner() {
+        return new ITuplePartitionComputer() {
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nPartitions)
+                    throws HyracksDataException {
+                int partitionI = getIntervalPartitionI(accessor, tIndex, intervalFieldId);
+                int partitionJ = getIntervalPartitionJ(accessor, tIndex, intervalFieldId);
+                return IntervalPartitionUtil.intervalPartitionMap(partitionI, partitionJ, k);
+            }
+
+            private int getIntervalPartition(long point) throws HyracksDataException {
+                return IntervalPartitionUtil.getIntervalPartition(point, partitionStart, partitionDuration, k);
+            }
+
+            public int getIntervalPartitionI(IFrameTupleAccessor accessor, int tIndex, int fieldId)
+                    throws HyracksDataException {
+                return getIntervalPartition(IntervalJoinUtil.getIntervalStart(accessor, tIndex, fieldId));
+            }
+
+            public int getIntervalPartitionJ(IFrameTupleAccessor accessor, int tIndex, int fieldId)
+                    throws HyracksDataException {
+                return getIntervalPartition(IntervalJoinUtil.getIntervalEnd(accessor, tIndex, fieldId));
+            }
+
+        };
+    }
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinOperatorDescriptor.java
new file mode 100644
index 0000000..ddbe913
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinOperatorDescriptor.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.intervalpartition;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinChecker;
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinCheckerFactory;
+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.ITuplePartitionComputer;
+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.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.join.MergeBranchStatus.Stage;
+import org.apache.hyracks.dataflow.std.join.MergeJoinLocks;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+
+public class IntervalPartitionJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int LEFT_ACTIVITY_ID = 0;
+    private static final int RIGHT_ACTIVITY_ID = 1;
+    private final int[] leftKeys;
+    private final int[] rightKeys;
+    private final int memoryForJoin;
+    private final IIntervalMergeJoinCheckerFactory imjcf;
+    private final RangeId rangeId;
+    private final int k;
+
+    private final int probeKey;
+    private final int buildKey;
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalPartitionJoinOperatorDescriptor.class.getName());
+
+    public IntervalPartitionJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin, int k,
+            int[] leftKeys, int[] rightKeys, RecordDescriptor recordDescriptor, IIntervalMergeJoinCheckerFactory imjcf,
+            RangeId rangeId) {
+        super(spec, 2, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.buildKey = leftKeys[0];
+        this.probeKey = rightKeys[0];
+        this.k = k;
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.memoryForJoin = memoryForJoin;
+        this.imjcf = imjcf;
+        this.rangeId = rangeId;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        MergeJoinLocks locks = new MergeJoinLocks();
+
+        ActivityId leftAid = new ActivityId(odId, LEFT_ACTIVITY_ID);
+        ActivityId rightAid = new ActivityId(odId, RIGHT_ACTIVITY_ID);
+
+        IActivity leftAN = new LeftJoinerActivityNode(leftAid, rightAid, locks);
+        IActivity rightAN = new RightDataActivityNode(rightAid, leftAid, locks);
+
+        builder.addActivity(this, rightAN);
+        builder.addSourceEdge(1, rightAN, 0);
+
+        builder.addActivity(this, leftAN);
+        builder.addSourceEdge(0, leftAN, 0);
+        builder.addTargetEdge(0, leftAN, 0);
+    }
+
+    private class LeftJoinerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final MergeJoinLocks locks;
+
+        public LeftJoinerActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new LeftJoinerOperator(ctx, partition, inRecordDesc);
+        }
+
+        private class LeftJoinerOperator extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+            private final IHyracksTaskContext ctx;
+            private final int partition;
+            private final RecordDescriptor leftRd;
+            private IntervalPartitionJoinTaskState state;
+            private boolean first = true;
+
+            public LeftJoinerOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.leftRd = inRecordDesc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    writer.open();
+                    state = new IntervalPartitionJoinTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));;
+                    state.leftRd = leftRd;
+                    ctx.setStateObject(state);
+                    locks.getRight(partition).signal();
+
+                    do {
+                        // Continue after joiner created in right branch.
+                        if (state.partitionJoiner == null) {
+                            locks.getLeft(partition).await();
+                        }
+                    } while (state.partitionJoiner == null);
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageOpen();
+                    locks.getRight(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+                if (first) {
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    state.partitionJoiner.setFrame(LEFT_ACTIVITY_ID, buffer);
+                    state.partitionJoiner.processLeftFrame(writer);
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[LEFT_ACTIVITY_ID].noMore();
+                    if (state.failed) {
+                        writer.fail();
+                    } else {
+                        state.partitionJoiner.processLeftClose(writer);
+                        writer.close();
+                    }
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageClose();
+                    locks.getRight(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+        }
+    }
+
+    private class RightDataActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId joinAid;
+        private final MergeJoinLocks locks;
+
+        public RightDataActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.joinAid = joinAid;
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new RightDataOperator(ctx, partition, inRecordDesc);
+        }
+
+        private class RightDataOperator extends AbstractUnaryInputSinkOperatorNodePushable {
+
+            private int partition;
+            private IHyracksTaskContext ctx;
+            private final RecordDescriptor rightRd;
+            private IntervalPartitionJoinTaskState state;
+            private boolean first = true;
+
+            public RightDataOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.rightRd = inRecordDesc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    do {
+                        // Wait for the state to be set in the context form Left.
+                        state = (IntervalPartitionJoinTaskState) ctx.getStateObject(new TaskId(joinAid, partition));
+                        if (state == null) {
+                            locks.getRight(partition).await();
+                        }
+                    } while (state == null);
+                    state.k = k;
+
+                    RangeForwardTaskState rangeState = RangeForwardTaskState.getRangeState(rangeId.getId(), ctx);
+                    long partitionStart = IntervalPartitionUtil.getStartOfPartition(rangeState.getRangeMap(),
+                            partition);
+                    long partitionEnd = IntervalPartitionUtil.getEndOfPartition(rangeState.getRangeMap(), partition);
+                    ITuplePartitionComputer buildHpc = new IntervalPartitionComputerFactory(buildKey, state.k,
+                            partitionStart, partitionEnd).createPartitioner();
+                    ITuplePartitionComputer probeHpc = new IntervalPartitionComputerFactory(probeKey, state.k,
+                            partitionStart, partitionEnd).createPartitioner();
+                    IIntervalMergeJoinChecker imjc = imjcf.createMergeJoinChecker(leftKeys, rightKeys, partition, ctx);
+
+                    state.rightRd = rightRd;
+                    state.partitionJoiner = new IntervalPartitionJoiner(ctx, memoryForJoin, partition, state.k,
+                            state.status, locks, imjc, state.leftRd, state.rightRd, buildHpc, probeHpc);
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageOpen();
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+                if (first) {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    while (!state.status.continueRightLoad
+                            && state.status.branch[LEFT_ACTIVITY_ID].getStatus() != Stage.CLOSED) {
+                        // Wait for the state to request right frame unless left has finished.
+                        locks.getRight(partition).await();
+                    }
+                    state.partitionJoiner.setFrame(RIGHT_ACTIVITY_ID, buffer);
+                    state.status.continueRightLoad = false;
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageClose();
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinTaskState.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinTaskState.java
index 500ac03..1939899 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoinTaskState.java
@@ -16,11 +16,18 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.asterix.runtime.operators.joins.intervalpartition;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.std.join.MergeJoinTaskState;
+
+public class IntervalPartitionJoinTaskState extends MergeJoinTaskState {
+    protected IntervalPartitionJoiner partitionJoiner;
+    public int k;
+
+    public IntervalPartitionJoinTaskState(JobId jobId, TaskId taskId) {
+        super(jobId, taskId);
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoiner.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoiner.java
new file mode 100644
index 0000000..f57d205
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionJoiner.java
@@ -0,0 +1,305 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.intervalpartition;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.TreeMap;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.asterix.runtime.operators.joins.IIntervalMergeJoinChecker;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+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.ITuplePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.buffermanager.BufferInfo;
+import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VPartitionTupleBufferManager;
+import org.apache.hyracks.dataflow.std.join.AbstractMergeJoiner;
+import org.apache.hyracks.dataflow.std.join.MergeJoinLocks;
+import org.apache.hyracks.dataflow.std.join.MergeStatus;
+import org.apache.hyracks.dataflow.std.structures.RunFilePointer;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class IntervalPartitionJoiner extends AbstractMergeJoiner {
+
+    private static final Logger LOGGER = Logger.getLogger(IntervalPartitionJoiner.class.getName());
+
+    private final RunFileWriter probeRunFileWriter;
+    private int probeRunFilePid = -1;
+
+    private final ITuplePartitionComputer buildHpc;
+    private final ITuplePartitionComputer probeHpc;
+
+    private final int buildMemory;
+    private final int k;
+    private final int numOfPartitions;
+    private long buildSize = 0;
+    private long probeSize = 0;
+    private long[] buildPartitionSizes;
+    private long[] probePartitionSizes;
+    private final TreeMap<RunFilePointer, Integer> probeRunFilePointers;
+
+    private final VPartitionTupleBufferManager buildBufferManager;
+    private final TuplePointer tempPtr = new TuplePointer();
+    private final List<Integer> buildInMemoryPartitions;
+    private final FrameTupleAccessor accessorBuild;
+    private BufferInfo bufferInfo;
+
+    private long spillWriteCount = 0;
+    private long spillReadCount = 0;
+    private long joinComparisonCount = 0;
+    private long joinResultCount = 0;
+    private final IIntervalMergeJoinChecker imjc;
+    private final FrameTupleAccessor accessorProbe;
+    private final IFrame reloadBuffer;
+    private boolean moreBuildProcessing = true;
+    private final List<IFrameBufferManager> fbms = new ArrayList<>();
+
+    public IntervalPartitionJoiner(IHyracksTaskContext ctx, int memorySize, int partition, int k, MergeStatus status,
+            MergeJoinLocks locks, IIntervalMergeJoinChecker imjc, RecordDescriptor leftRd, RecordDescriptor rightRd,
+            ITuplePartitionComputer buildHpc, ITuplePartitionComputer probeHpc) throws HyracksDataException {
+        super(ctx, partition, status, locks, leftRd, rightRd);
+
+        bufferInfo = new BufferInfo(null, -1, -1);
+
+        this.accessorProbe = new FrameTupleAccessor(leftRd);
+        reloadBuffer = new VSizeFrame(ctx);
+
+        this.numOfPartitions = IntervalPartitionUtil.getMaxPartitions(k);
+        this.imjc = imjc;
+        buildPartitionSizes = new long[numOfPartitions];
+        probePartitionSizes = new long[numOfPartitions];
+
+        // TODO fix available memory size
+        this.buildMemory = memorySize;
+        buildBufferManager = new VPartitionTupleBufferManager(ctx, VPartitionTupleBufferManager.NO_CONSTRAIN,
+                numOfPartitions, buildMemory * ctx.getInitialFrameSize());
+
+        this.k = k;
+        this.buildHpc = buildHpc;
+        this.probeHpc = probeHpc;
+
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile("IntervalPartitionJoiner");
+        probeRunFileWriter = new RunFileWriter(file, ctx.getIOManager());
+        probeRunFileWriter.open();
+
+        probeRunFilePointers = new TreeMap<>(RunFilePointer.ASC);
+        buildInMemoryPartitions = new LinkedList<>();
+
+        this.accessorBuild = new FrameTupleAccessor(rightRd);
+
+        LOGGER.setLevel(Level.FINE);
+        System.out.println("IntervalIndexJoiner: Logging level is: " + LOGGER.getLevel());
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("IntervalIndexJoiner has started partition " + partition + " with " + memorySize
+                    + " frames of memory.");
+        }
+    }
+
+    @Override
+    public void processLeftFrame(IFrameWriter writer) throws HyracksDataException {
+        while (inputAccessor[LEFT_PARTITION].exists()) {
+            int pid = probeHpc.partition(inputAccessor[LEFT_PARTITION], inputAccessor[LEFT_PARTITION].getTupleId(), k);
+
+            if (probeRunFilePid != pid) {
+                // Log new partition locations.
+                RunFilePointer rfp = new RunFilePointer(probeRunFileWriter.getFileSize(),
+                        inputAccessor[LEFT_PARTITION].getTupleId());
+                probeRunFilePointers.put(rfp, pid);
+                probeRunFilePid = pid;
+            }
+            inputAccessor[LEFT_PARTITION].next();
+            probeSize++;
+            probePartitionSizes[pid]++;
+        }
+        inputBuffer[LEFT_PARTITION].rewind();
+        probeRunFileWriter.nextFrame(inputBuffer[LEFT_PARTITION]);
+        spillWriteCount++;
+    }
+
+    @Override
+    public void processLeftClose(IFrameWriter writer) throws HyracksDataException {
+        joinLoopOnMemory(writer);
+
+        // Flush result.
+        resultAppender.write(writer, true);
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("IntervalPartitionJoiner statitics: " + k + " k, " + joinComparisonCount + " comparisons, "
+                    + joinResultCount + " results, " + spillWriteCount + " written, " + spillReadCount + " read.");
+        }
+        System.err.print("build: [");
+        for (int i = 0; i < buildPartitionSizes.length; i++) {
+            System.err.print(buildPartitionSizes[i] + ", ");
+        }
+        System.err.println("]");
+        System.err.print("probe: [");
+        for (int i = 0; i < probePartitionSizes.length; i++) {
+            System.err.print(probePartitionSizes[i] + ", ");
+        }
+        System.err.println("]");
+    }
+
+    private void joinLoopOnMemory(IFrameWriter writer) throws HyracksDataException {
+        RunFileReader pReader = probeRunFileWriter.createDeleteOnCloseReader();
+        pReader.open();
+        // Load first frame.
+        loadReaderNextFrame(pReader);
+
+        while (moreBuildProcessing) {
+            fillMemory();
+            joinMemoryBlockWithRunFile(writer, pReader);
+
+            // Clean up
+            for (int pid : buildInMemoryPartitions) {
+                buildBufferManager.clearPartition(pid);
+            }
+            buildInMemoryPartitions.clear();
+        }
+        pReader.close();
+    }
+
+    private void joinMemoryBlockWithRunFile(IFrameWriter writer, RunFileReader pReader) throws HyracksDataException {
+        // Join Disk partitions with Memory partitions
+        for (RunFilePointer probeId : probeRunFilePointers.navigableKeySet()) {
+            Pair<Integer, Integer> probe = IntervalPartitionUtil.getIntervalPartition(probeRunFilePointers.get(probeId),
+                    k);
+            for (int buildId : buildInMemoryPartitions) {
+                Pair<Integer, Integer> build = IntervalPartitionUtil.getIntervalPartition(buildId, k);
+                if (imjc.compareIntervalPartition(probe.first, probe.second, build.first, build.second)) {
+                    fbms.add(buildBufferManager.getPartitionFrameBufferManager(buildId));
+                    System.err.println("join " + probe + "(" + probePartitionSizes[probeRunFilePointers.get(probeId)]
+                            + ") with " + build + "(" + buildPartitionSizes[buildId] + ")");
+                }
+            }
+            if (!fbms.isEmpty()) {
+                join(pReader, probeId, fbms, writer);
+            }
+            fbms.clear();
+        }
+    }
+
+    private void join(RunFileReader pReader, RunFilePointer rfpStart, List<IFrameBufferManager> buildFbms,
+            IFrameWriter writer) throws HyracksDataException {
+        long fileOffsetStart = rfpStart.getFileOffset();
+        int tupleStart = rfpStart.getTupleIndex();
+
+        RunFilePointer rfpEnd = probeRunFilePointers.higherKey(rfpStart);
+        long fileOffsetEnd = rfpEnd == null ? pReader.getFileSize() : rfpEnd.getFileOffset();
+        int tupleEnd = rfpEnd == null ? Integer.MAX_VALUE : rfpEnd.getTupleIndex();
+
+        if (pReader.getReadPointer() != fileOffsetStart) {
+            pReader.reset(fileOffsetStart);
+            loadReaderNextFrame(pReader);
+        }
+        do {
+            int start = pReader.getReadPointer() == fileOffsetStart ? tupleStart : 0;
+            int end = pReader.getReadPointer() == fileOffsetEnd ? tupleEnd : accessorProbe.getTupleCount();
+
+            for (int i = start; i < end; ++i) {
+                // Tuple has potential match from build phase
+                for (IFrameBufferManager fbm : buildFbms) {
+                    joinTupleWithMemoryPartition(accessorProbe, i, fbm, writer);
+                }
+            }
+        } while (pReader.getReadPointer() < fileOffsetEnd && loadReaderNextFrame(pReader));
+    }
+
+    private boolean loadReaderNextFrame(RunFileReader pReader) throws HyracksDataException {
+        if (pReader.nextFrame(reloadBuffer)) {
+            accessorProbe.reset(reloadBuffer.getBuffer());
+            spillReadCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public void joinTupleWithMemoryPartition(IFrameTupleAccessor accessorProbe, int probeTupleIndex,
+            IFrameBufferManager fbm, IFrameWriter writer) throws HyracksDataException {
+        if (fbm.getNumFrames() == 0) {
+            return;
+        }
+        fbm.resetIterator();
+        int frameIndex = fbm.next();
+        while (fbm.exists()) {
+            fbm.getFrame(frameIndex, bufferInfo);
+            accessorBuild.reset(bufferInfo.getBuffer());
+            for (int buildTupleIndex = 0; buildTupleIndex < accessorBuild.getTupleCount(); ++buildTupleIndex) {
+                if (imjc.checkToSaveInResult(accessorBuild, buildTupleIndex, accessorProbe, probeTupleIndex, false)) {
+                    appendToResult(accessorBuild, buildTupleIndex, accessorProbe, probeTupleIndex, writer);
+                }
+                joinComparisonCount++;
+            }
+            frameIndex = fbm.next();
+        }
+    }
+
+    private void appendToResult(IFrameTupleAccessor accessorBuild, int buildSidetIx, IFrameTupleAccessor accessorProbe,
+            int probeSidetIx, IFrameWriter writer) throws HyracksDataException {
+        FrameUtils.appendConcatToWriter(writer, resultAppender, accessorBuild, buildSidetIx, accessorProbe,
+                probeSidetIx);
+        joinResultCount++;
+    }
+
+    private void fillMemory() throws HyracksDataException {
+        int buildPid = -1;
+        TupleStatus ts;
+        for (ts = loadRightTuple(); ts.isLoaded(); ts = loadRightTuple()) {
+            int pid = buildHpc.partition(inputAccessor[RIGHT_PARTITION], inputAccessor[RIGHT_PARTITION].getTupleId(),
+                    k);
+            if (!buildBufferManager.insertTuple(pid, inputAccessor[RIGHT_PARTITION],
+                    inputAccessor[RIGHT_PARTITION].getTupleId(), tempPtr)) {
+                return;
+            }
+            buildPartitionSizes[pid]++;
+
+            if (buildPid != pid) {
+                // Track new partitions in memory.
+                buildInMemoryPartitions.add(pid);
+                buildPid = pid;
+            }
+            inputAccessor[RIGHT_PARTITION].next();
+            buildSize++;
+        }
+        if (ts.isEmpty()) {
+            moreBuildProcessing = false;
+        }
+    }
+
+    private TupleStatus loadRightTuple() throws HyracksDataException {
+        TupleStatus loaded = loadMemoryTuple(RIGHT_PARTITION);
+        if (loaded == TupleStatus.UNKNOWN) {
+            loaded = pauseAndLoadRightTuple();
+        }
+        return loaded;
+    }
+
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionUtil.java
new file mode 100644
index 0000000..671c082
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/intervalpartition/IntervalPartitionUtil.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.intervalpartition;
+
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+
+public class IntervalPartitionUtil {
+    public static final double C_CPU = 0.5;
+    public static final double C_IO = 100000;
+    public static final int ITERATION_LIMIT = 20;
+
+    private IntervalPartitionUtil() {
+    }
+
+    public static void main(String[] args) {
+        PhysicalOptimizationConfig poc = new PhysicalOptimizationConfig();
+        long[] countList = { poc.getMaxFramesForJoinLeftInput(), 2441, 9766, 39063, 156250, 625000, 2500000, 10000000 };
+        long[] maxDurationList = { poc.getMaxIntervalDuration(), 1, 3, 30, 300, 3000, 30000, 300000 };
+        int[] tuplesList = { poc.getMaxRecordsPerFrame(), 5, 50, 300, 900 };
+
+        int k;
+        for (long count : countList) {
+            for (long maxDuration : maxDurationList) {
+                for (int tuples : tuplesList) {
+                    k = determineK(count, maxDuration, count, maxDuration, tuples);
+                    System.err.println(
+                            "size: " + count + " duration: " + maxDuration + " tuples: " + tuples + " k: " + k);
+                }
+            }
+        }
+    }
+
+    public static int determineK(long countR, long maxDurationR, long countS, long maxDurationS, int avgTuplePerFrame) {
+        double deltaR = 1.0 / maxDurationR;
+        double deltaS = 1.0 / maxDurationS;
+
+        long knMinusTwo = 0;
+        long knMinusOne = 0;
+        long kn = 1;
+
+        long prn = determinePn(kn, countR, deltaR);
+        double tn = determineTn(kn, determinePn(kn, countS, deltaS));
+
+        int count = 0;
+        while ((kn != knMinusOne) && (kn != knMinusTwo) && count < ITERATION_LIMIT) {
+            knMinusTwo = knMinusOne;
+            knMinusOne = kn;
+            kn = determineKn(countR, countS, avgTuplePerFrame, prn, tn);
+            prn = determinePn(kn, countR, deltaR);
+            tn = determineTn(kn, determinePn(kn, countS, deltaS));
+            count++;
+        }
+        if (count == ITERATION_LIMIT) {
+            kn = (kn + knMinusOne + knMinusTwo) / 3;
+        } else if (kn == knMinusTwo) {
+            kn = (kn + knMinusTwo) / 2;
+        }
+        if (kn > Integer.MAX_VALUE) {
+            return Integer.MAX_VALUE;
+        } else {
+            return (int) kn;
+        }
+    }
+
+    public static long determineKn(long countR, long countS, int avgTuplePerFrame, long prn, double tn) {
+        double factorS = (3.0 * countS) / (2 * (C_IO + 2 * C_CPU) * tn);
+        double factorR = (C_IO / avgTuplePerFrame) + ((4.0 * countR * C_CPU) / prn);
+        return (long) Math.cbrt(factorS * factorR);
+    }
+
+    public static long determinePn(long kn, long count, double delta) {
+        double knDelta = Math.ceil(kn * delta);
+        return Math.min((long) ((kn * knDelta) + kn - ((knDelta * knDelta) / 2.0) - (knDelta / 2.0)), count);
+    }
+
+    public static double determineTn(long kn, long pn) {
+        return pn / ((kn * kn + kn) / 2.0);
+    }
+
+    public static int getMaxPartitions(int k) {
+        return (k * k + k) / 2;
+    }
+
+    /**
+     * Map the partition start and end points to a single value.
+     * The mapped partitions are sorted in interval starting at 0.
+     *
+     * @param partitionI
+     *            start point
+     * @param partitionJ
+     *            end point
+     * @param k
+     *            granules
+     * @return mapping
+     */
+    public static int intervalPartitionMap(int partitionI, int partitionJ, int k) {
+        int p = (partitionI * (k + k - partitionI + 1)) / 2;
+        return p + partitionJ - partitionI;
+    }
+
+    /**
+     * Reverse the map to individual start and end points.
+     *
+     * @param i
+     *            map id
+     * @param k
+     *            granules
+     * @return start and end points
+     */
+    public static Pair<Integer, Integer> getIntervalPartition(int pid, int k) {
+        int i = 0;
+        int sum = 0;
+        for (int p = k; p <= pid; p += k - i) {
+            ++i;
+            sum = p;
+        }
+        int j = i + pid - sum;
+        return new Pair<>(i, j);
+    }
+
+    public static long getStartOfPartition(IRangeMap rangeMap, int partition) {
+        int fieldIndex = 0;
+        long partitionStart = LongPointable.getLong(rangeMap.getMinByteArray(fieldIndex),
+                rangeMap.getMinStartOffset(fieldIndex) + 1);
+        if (partition != 0 && partition <= rangeMap.getSplitCount()) {
+            partitionStart = LongPointable.getLong(rangeMap.getByteArray(fieldIndex, partition - 1),
+                    rangeMap.getStartOffset(fieldIndex, partition - 1) + 1);
+        } else if (partition > rangeMap.getSplitCount()) {
+            partitionStart = LongPointable.getLong(rangeMap.getMaxByteArray(fieldIndex),
+                    rangeMap.getMaxStartOffset(fieldIndex) + 1);
+        }
+        return partitionStart;
+    }
+
+    public static long getEndOfPartition(IRangeMap rangeMap, int partition) {
+        int fieldIndex = 0;
+        long partitionEnd = LongPointable.getLong(rangeMap.getMaxByteArray(fieldIndex),
+                rangeMap.getMaxStartOffset(fieldIndex) + 1);
+        if (partition < rangeMap.getSplitCount()) {
+            partitionEnd = LongPointable.getLong(rangeMap.getByteArray(fieldIndex, partition),
+                    rangeMap.getStartOffset(fieldIndex, partition) + 1);
+        }
+        return partitionEnd;
+    }
+
+    public static long getPartitionDuration(long partitionStart, long partitionEnd, int k) throws HyracksDataException {
+        if (k <= 2) {
+            throw new HyracksDataException("k is to small for interval partitioner.");
+        }
+        long duration = (partitionEnd - partitionStart) / (k - 2);
+        if (duration <= 0) {
+            duration = 1;
+        }
+        return duration;
+    }
+
+    public static int getIntervalPartition(long point, long partitionStart, long partitionDuration, int k)
+            throws HyracksDataException {
+        if (point < partitionStart) {
+            return 0;
+        }
+        long pointFloor = Math.floorDiv(point - partitionStart, partitionDuration);
+        // Add one to the partition, since 0 represents any point before the start partition point.
+        return (int) Math.min(pointFloor + 1, k - 1L);
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
index 328697d..dfd36a2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IOptimizationContext.java
@@ -33,7 +33,7 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 
-public interface IOptimizationContext extends ITypingContext, IVariableContext {
+public interface IOptimizationContext extends ITypingContext, IVariableContext, IRangeContext {
 
     @Override
     public IMetadataProvider<?, ?> getMetadataProvider();
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IRangeContext.java
similarity index 80%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql
copy to hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IRangeContext.java
index f686b54..9a70b7a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/IRangeContext.java
@@ -16,7 +16,12 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/**
- * overlap_bins test case: test the overlap_bins
- * Expected result: success
- **/
+package org.apache.hyracks.algebricks.core.algebra.base;
+
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public interface IRangeContext {
+
+    public RangeId newRangeId();
+
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index cc7a75f..c7f9aa1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -39,6 +39,7 @@
     NESTEDTUPLESOURCE,
     ORDER,
     PROJECT,
+    RANGE_FORWARD,
     REPLICATE,
     RUNNINGAGGREGATE,
     SCRIPT,
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 1d20e08..949f6c8 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
@@ -42,6 +42,7 @@
     INSERT_DELETE,
     LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
     MATERIALIZE,
+    MERGE_JOIN,
     MICRO_PRE_CLUSTERED_GROUP_BY,
     NESTED_LOOP,
     NESTED_TUPLE_SOURCE,
@@ -50,6 +51,7 @@
     PRE_SORTED_DISTINCT_BY,
     RANDOM_PARTITION_EXCHANGE,
     RANDOM_MERGE_EXCHANGE,
+    RANGE_FORWARD,
     RANGE_PARTITION_EXCHANGE,
     RANGE_PARTITION_MERGE_EXCHANGE,
     REPLICATE,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RangeForwardOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RangeForwardOperator.java
new file mode 100644
index 0000000..58c7a80
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/RangeForwardOperator.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.logical;
+
+import java.util.ArrayList;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public class RangeForwardOperator extends AbstractLogicalOperator {
+
+    private RangeId rangeId;
+    private IRangeMap rangeMap;
+
+    public RangeForwardOperator(RangeId rangeId, IRangeMap rangeMap) {
+        this.rangeId = rangeId;
+        this.rangeMap = rangeMap;
+    }
+
+    public RangeId getRangeId() {
+        return rangeId;
+    }
+
+    public IRangeMap getRangeMap() {
+        return rangeMap;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.RANGE_FORWARD;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return VariablePropagationPolicy.ALL;
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitRangeForwardOperator(this, arg);
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        return createPropagatingAllInputsTypeEnvironment(ctx);
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public void recomputeSchema() throws AlgebricksException {
+        schema = new ArrayList<LogicalVariable>(inputs.get(0).getValue().getSchema());
+    }
+
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index d278078..1ab0606 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -49,6 +49,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -185,6 +186,11 @@
     }
 
     @Override
+    public Long visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return op.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    @Override
     public Long visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return op.getInputs().get(0).getValue().accept(this, arg);
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index b259869..2bb43c9 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -67,6 +67,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -404,6 +405,12 @@
     }
 
     @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        throw new NotImplementedException();
+    }
+
+    @Override
     public Void visitProjectOperator(ProjectOperator op, IOptimizationContext ctx) throws AlgebricksException {
         propagateFDsAndEquivClassesForUsedVars(op, ctx, op.getVariables());
         return null;
@@ -416,7 +423,7 @@
     }
 
     @Override
-    public Void visitSplitOperator(SplitOperator op, IOptimizationContext ctx) throws AlgebricksException {
+    public Void visitRangeForwardOperator(RangeForwardOperator op, IOptimizationContext ctx) throws AlgebricksException {
         propagateFDsAndEquivClasses(op, ctx);
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 7f34e8b..7c114c8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -56,6 +56,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -297,6 +298,15 @@
     }
 
     @Override
+    public Boolean visitRangeForwardOperator(RangeForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+        if (aop.getOperatorTag() != LogicalOperatorTag.RANGE_FORWARD) {
+            return Boolean.FALSE;
+        }
+        return Boolean.TRUE;
+    }
+
+    @Override
     public Boolean visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
         if (aop.getOperatorTag() != LogicalOperatorTag.MATERIALIZE) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 58b31f8..7f099cb 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -58,6 +58,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -177,6 +178,12 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+   @Override
     public Void visitMaterializeOperator(MaterializeOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index f4b3195..d68f8d7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -54,6 +54,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -402,6 +403,14 @@
     }
 
     @Override
+    public ILogicalOperator visitRangeForwardOperator(RangeForwardOperator op, ILogicalOperator arg)
+            throws AlgebricksException {
+        // TODO fix deep copy of range map
+        RangeForwardOperator opCopy = new RangeForwardOperator(op.getRangeId(), op.getRangeMap());
+        deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
+        return opCopy;
+    }
+
     public ILogicalOperator visitSplitOperator(SplitOperator op, ILogicalOperator arg) throws AlgebricksException {
         SplitOperator opCopy = new SplitOperator(op.getOutputArity(), op.getBranchingExpression());
         deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 7e92869..f7113c0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -47,7 +47,9 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.PartitioningSplitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -174,6 +176,13 @@
     }
 
     @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, IOptimizationContext arg)
+            throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
     public Void visitProjectOperator(ProjectOperator op, IOptimizationContext context) throws AlgebricksException {
         propagateCardinalityAndFrameNumber(op, context);
         return null;
@@ -186,6 +195,12 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, IOptimizationContext arg) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
     public Void visitSplitOperator(SplitOperator op, IOptimizationContext arg) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 442899f..578dab0 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
@@ -54,6 +55,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -291,9 +293,9 @@
         deepCopyExpressionRefs(newKeyExpressions, op.getPrimaryKeyExpressions());
         List<Mutable<ILogicalExpression>> newLSMComponentFilterExpressions = new ArrayList<>();
         deepCopyExpressionRefs(newKeyExpressions, op.getAdditionalFilteringExpressions());
-        InsertDeleteUpsertOperator insertDeleteOp =
-                new InsertDeleteUpsertOperator(op.getDataSource(), deepCopyExpressionRef(op.getPayloadExpression()),
-                        newKeyExpressions, op.getOperation(), op.isBulkload());
+        InsertDeleteUpsertOperator insertDeleteOp = new InsertDeleteUpsertOperator(op.getDataSource(),
+                deepCopyExpressionRef(op.getPayloadExpression()), newKeyExpressions, op.getOperation(),
+                op.isBulkload());
         insertDeleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
         return insertDeleteOp;
     }
@@ -305,8 +307,8 @@
         deepCopyExpressionRefs(newPrimaryKeyExpressions, op.getPrimaryKeyExpressions());
         List<Mutable<ILogicalExpression>> newSecondaryKeyExpressions = new ArrayList<>();
         deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
-        Mutable<ILogicalExpression> newFilterExpression =
-                new MutableObject<>(((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
+        Mutable<ILogicalExpression> newFilterExpression = new MutableObject<>(
+                ((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
         List<Mutable<ILogicalExpression>> newLSMComponentFilterExpressions = new ArrayList<>();
         deepCopyExpressionRefs(newLSMComponentFilterExpressions, op.getAdditionalFilteringExpressions());
         IndexInsertDeleteUpsertOperator indexInsertDeleteOp = new IndexInsertDeleteUpsertOperator(
@@ -324,8 +326,8 @@
         deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
         List<LogicalVariable> newTokenizeVars = new ArrayList<>();
         deepCopyVars(newTokenizeVars, op.getTokenizeVars());
-        Mutable<ILogicalExpression> newFilterExpression =
-                new MutableObject<>(((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
+        Mutable<ILogicalExpression> newFilterExpression = new MutableObject<>(
+                ((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
         List<Object> newTokenizeVarTypes = new ArrayList<>();
         deepCopyObjects(newTokenizeVarTypes, op.getTokenizeVarTypes());
 
@@ -369,8 +371,8 @@
         return newObjs;
     }
 
-    private List<Pair<IOrder, Mutable<ILogicalExpression>>>
-            deepCopyOrderAndExpression(List<Pair<IOrder, Mutable<ILogicalExpression>>> ordersAndExprs) {
+    private List<Pair<IOrder, Mutable<ILogicalExpression>>> deepCopyOrderAndExpression(
+            List<Pair<IOrder, Mutable<ILogicalExpression>>> ordersAndExprs) {
         List<Pair<IOrder, Mutable<ILogicalExpression>>> newOrdersAndExprs = new ArrayList<>();
         for (Pair<IOrder, Mutable<ILogicalExpression>> pair : ordersAndExprs) {
             newOrdersAndExprs.add(new Pair<>(pair.first, deepCopyExpressionRef(pair.second)));
@@ -384,6 +386,11 @@
     }
 
     @Override
+    public ILogicalOperator visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return new RangeForwardOperator(op.getRangeId(), op.getRangeMap());
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return new MaterializeOperator();
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index 9f1acea..ec74b4e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -184,6 +185,12 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, IOptimizationContext ctx)
+            throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, IOptimizationContext ctx) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 3645aff..a5adfa4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -54,6 +54,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -250,6 +251,11 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index a746cf2..e1343d3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -52,6 +52,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -282,6 +283,12 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
     public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
         standardLayout(op);
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 7345928..5d6a7b3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -53,6 +53,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -219,6 +220,16 @@
     }
 
     @Override
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
     public Void visitProjectOperator(ProjectOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         List<LogicalVariable> usedVariables = op.getVariables();
@@ -411,6 +422,12 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Pair<LogicalVariable, LogicalVariable> arg)
             throws AlgebricksException {
         return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 3daa00f..e6a8032 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
@@ -53,6 +53,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -438,6 +439,11 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return null;
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
new file mode 100644
index 0000000..7fa7fdf
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+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.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+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.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+import org.apache.hyracks.dataflow.std.join.MergeJoinOperatorDescriptor;
+
+public class MergeJoinPOperator extends AbstractJoinPOperator {
+
+    private final int memSizeInFrames;
+    protected final List<LogicalVariable> keysLeftBranch;
+    protected final List<LogicalVariable> keysRightBranch;
+    private final IMergeJoinCheckerFactory mjcf;
+    private final RangeId leftRangeId;
+    private final RangeId rightRangeId;
+    private final IRangeMap rangeMapHint;
+
+    private static final Logger LOGGER = Logger.getLogger(MergeJoinPOperator.class.getName());
+
+    public MergeJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType, List<LogicalVariable> sideLeft,
+            List<LogicalVariable> sideRight, int memSizeInFrames, IMergeJoinCheckerFactory mjcf, RangeId leftRangeId,
+            RangeId rightRangeId, IRangeMap rangeMapHint) {
+        super(kind, partitioningType);
+        this.memSizeInFrames = memSizeInFrames;
+        this.keysLeftBranch = sideLeft;
+        this.keysRightBranch = sideRight;
+        this.mjcf = mjcf;
+        this.leftRangeId = leftRangeId;
+        this.rightRangeId = rightRangeId;
+        this.rangeMapHint = rangeMapHint;
+
+        LOGGER.fine("MergeJoinPOperator constructed with: JoinKind=" + kind + ", JoinPartitioningType="
+                + partitioningType + ", List<LogicalVariable>=" + keysLeftBranch + ", List<LogicalVariable>="
+                + keysRightBranch + ", int memSizeInFrames=" + memSizeInFrames + ", IMergeJoinCheckerFactory mjcf="
+                + mjcf + ", RangeId leftRangeId=" + leftRangeId + ", RangeId rightRangeId=" + rightRangeId + ".");
+    }
+
+    public List<LogicalVariable> getKeysLeftBranch() {
+        return keysLeftBranch;
+    }
+
+    public List<LogicalVariable> getKeysRightBranch() {
+        return keysRightBranch;
+    }
+
+    public IMergeJoinCheckerFactory getMergeJoinCheckerFactory() {
+        return mjcf;
+    }
+
+    public RangeId getLeftRangeId() {
+        return leftRangeId;
+    }
+
+    public RangeId getRightRangeId() {
+        return rightRangeId;
+    }
+
+    public IRangeMap getRangeMapHint() {
+        return rangeMapHint;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.MERGE_JOIN;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context) {
+        ArrayList<OrderColumn> order = new ArrayList<>();
+        for (LogicalVariable v : keysLeftBranch) {
+            order.add(new OrderColumn(v, mjcf.isOrderAsc() ? OrderKind.ASC : OrderKind.DESC));
+        }
+        IPartitioningProperty pp = new OrderedPartitionedProperty(order, null, leftRangeId,
+                RangePartitioningType.PROJECT, rangeMapHint);
+        List<ILocalStructuralProperty> propsLocal = new ArrayList<>();
+        propsLocal.add(new LocalOrderProperty(order));
+        deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+        IPartitioningProperty ppLeft = null;
+        List<ILocalStructuralProperty> ispLeft = new ArrayList<>();
+        IPartitioningProperty ppRight = null;
+        List<ILocalStructuralProperty> ispRight = new ArrayList<>();
+
+        ArrayList<OrderColumn> orderLeft = new ArrayList<>();
+        for (LogicalVariable v : keysLeftBranch) {
+            orderLeft.add(new OrderColumn(v, mjcf.isOrderAsc() ? OrderKind.ASC : OrderKind.DESC));
+        }
+        ispLeft.add(new LocalOrderProperty(orderLeft));
+
+        ArrayList<OrderColumn> orderRight = new ArrayList<>();
+        for (LogicalVariable v : keysRightBranch) {
+            orderRight.add(new OrderColumn(v, mjcf.isOrderAsc() ? OrderKind.ASC : OrderKind.DESC));
+        }
+        ispRight.add(new LocalOrderProperty(orderRight));
+
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+            ppLeft = new OrderedPartitionedProperty(orderLeft, null, leftRangeId, mjcf.getLeftPartitioningType(),
+                    rangeMapHint);
+            ppRight = new OrderedPartitionedProperty(orderRight, null, rightRangeId, mjcf.getRightPartitioningType(),
+                    rangeMapHint);
+        }
+
+        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);
+
+        MergeJoinOperatorDescriptor opDesc = new MergeJoinOperatorDescriptor(spec, memSizeInFrames, recordDescriptor,
+                keysLeft, keysRight, 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);
+    }
+
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangeForwardPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangeForwardPOperator.java
new file mode 100644
index 0000000..3ace793
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangeForwardPOperator.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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 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.PhysicalOperatorTag;
+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.properties.IPhysicalPropertiesVector;
+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.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor;
+
+public class RangeForwardPOperator extends AbstractPhysicalOperator {
+
+    private RangeId rangeId;
+    private IRangeMap rangeMap;
+
+    public RangeForwardPOperator(RangeId rangeId, IRangeMap rangeMap) {
+        // Use when a range hint is provided.
+        this.rangeId = rangeId;
+        this.rangeMap = rangeMap;
+    }
+
+    public RangeForwardPOperator(RangeId rangeId) {
+        this(rangeId, null);
+    }
+
+    public IRangeMap getRangeMap() {
+        return rangeMap;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.RANGE_FORWARD;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        IOperatorDescriptorRegistry spec = builder.getJobSpec();
+        RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+                context);
+        RangeForwardOperatorDescriptor opDesc = new RangeForwardOperatorDescriptor(spec, rangeId, rangeMap,
+                recordDescriptor);
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+
+    @Override
+    public boolean expensiveThanMaterialization() {
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return getOperatorTag().toString() + " " + rangeId;
+    }
+
+}
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 225ffa0..c58bb0c 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,6 @@
 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;
@@ -30,9 +29,11 @@
 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.IPhysicalPropertiesVector;
@@ -45,23 +46,28 @@
 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.IBinaryRangeComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
 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.IRangeMap;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.connectors.MToNRangePartitioningConnectorDescriptor;
 
 public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeId rangeId;
+    private RangePartitioningType rangeType;
 
-    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap) {
+    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, RangeId rangeId,
+            RangePartitioningType rangeType) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
-        this.rangeMap = rangeMap;
+        this.rangeId = rangeId;
+        this.rangeType = rangeType;
     }
 
     @Override
@@ -73,14 +79,33 @@
         return partitioningFields;
     }
 
+    public RangePartitioningType getRangeType() {
+        return rangeType;
+    }
+
+    public RangeId getRangeId() {
+        return rangeId;
+    }
+
     public INodeDomain getDomain() {
         return domain;
     }
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain);
-        this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
+        IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain,
+                rangeId, rangeType, null);
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
+        List<ILocalStructuralProperty> locals = new ArrayList<>();
+        for (ILocalStructuralProperty prop : op2Locals) {
+            if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+                locals.add(prop);
+            } else {
+                break;
+            }
+        }
+        this.deliveredProperties = new StructuralPropertiesVector(p, locals);
     }
 
     @Override
@@ -94,7 +119,8 @@
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
         int n = partitioningFields.size();
         int[] sortFields = new int[n];
-        IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+        IBinaryRangeComparatorFactory[] rangeComps = new IBinaryRangeComparatorFactory[n];
+        IBinaryComparatorFactory[] binaryComps = new IBinaryComparatorFactory[n];
 
         INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
         INormalizedKeyComputerFactory nkcf = null;
@@ -110,17 +136,20 @@
                 nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderKind.ASC);
             }
             IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
-            comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
+            rangeComps[i] = bcfp.getRangeBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC, rangeType);
+            binaryComps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
-        IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, tpcf);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+        ITupleRangePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, rangeComps,
+                rangeType);
+        IConnectorDescriptor conn = new MToNRangePartitioningConnectorDescriptor(spec, tpcf, rangeId, sortFields, binaryComps,
+                nkcf);
+        return new Pair<>(conn, null);
     }
 
     @Override
     public String toString() {
-        return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+        return getOperatorTag().toString() + " " + partitioningFields + " " + rangeType + " " + rangeId;
     }
 
 }
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
index f56a5dc..31cd37d 100644
--- 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
@@ -23,7 +23,6 @@
 import java.util.List;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.ListSet;
 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;
@@ -42,31 +41,36 @@
 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.algebra.properties.UnorderedPartitionedProperty;
 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.IBinaryRangeComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
 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.IRangeMap;
-import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.connectors.MToNRangePartitionMergingConnectorDescriptor;
 
 public class RangePartitionMergeExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeId rangeId;
+    private RangePartitioningType rangeType;
 
-    public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap) {
+    public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+            RangeId rangeId, RangePartitioningType rangeType) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
-        this.rangeMap = rangeMap;
+        this.rangeId = rangeId;
+        this.rangeType = rangeType;
     }
 
     @Override
@@ -78,20 +82,24 @@
         return partitioningFields;
     }
 
+    public RangePartitioningType getRangeType() {
+        return rangeType;
+    }
+
+    public RangeId getRangeId() {
+        return rangeId;
+    }
+
     public INodeDomain getDomain() {
         return domain;
     }
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
-        for (OrderColumn oc : partitioningFields) {
-            varList.add(oc.getColumn());
-        }
-        IPartitioningProperty p = new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(varList), domain);
+        IPartitioningProperty p = new OrderedPartitionedProperty(partitioningFields, domain, rangeId, rangeType, null);
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
         List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
-        List<ILocalStructuralProperty> locals = new ArrayList<ILocalStructuralProperty>();
+        List<ILocalStructuralProperty> locals = new ArrayList<>();
         for (ILocalStructuralProperty prop : op2Locals) {
             if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
                 locals.add(prop);
@@ -99,22 +107,21 @@
                 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>();
+        List<ILocalStructuralProperty> orderProps = new LinkedList<>();
+        List<OrderColumn> columns = new ArrayList<>();
         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) };
+        StructuralPropertiesVector[] r = new StructuralPropertiesVector[] {
+                new StructuralPropertiesVector(null, orderProps) };
         return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
     }
 
@@ -123,7 +130,8 @@
             ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
         int n = partitioningFields.size();
         int[] sortFields = new int[n];
-        IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
+        IBinaryRangeComparatorFactory[] rangeComps = new IBinaryRangeComparatorFactory[n];
+        IBinaryComparatorFactory[] binaryComps = new IBinaryComparatorFactory[n];
 
         INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
         INormalizedKeyComputerFactory nkcf = null;
@@ -139,17 +147,20 @@
                 nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderKind.ASC);
             }
             IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
-            comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
+            rangeComps[i] = bcfp.getRangeBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC, rangeType);
+            binaryComps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
-        IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+        ITupleRangePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, rangeComps,
+                rangeType);
+        IConnectorDescriptor conn = new MToNRangePartitionMergingConnectorDescriptor(spec, tpcf, rangeId, sortFields,
+                binaryComps, nkcf);
+        return new Pair<>(conn, null);
     }
 
     @Override
     public String toString() {
-        return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+        return getOperatorTag().toString() + " " + partitioningFields + " " + rangeType + " " + rangeId;
     }
 
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index d3dd166..566f954 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -51,6 +51,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -354,6 +355,13 @@
     }
 
     @Override
+    public Void visitRangeForwardOperator(RangeForwardOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent).append("range forward ");
+        addIndent(indent).append("split " + branchingExpression.getValue().accept(exprVisitor, indent));
+        return null;
+    }
+
+    @Override
     public Void visitSplitOperator(SplitOperator op, Integer indent) throws AlgebricksException {
         Mutable<ILogicalExpression> branchingExpression = op.getBranchingExpression();
         addIndent(indent).append("split " + branchingExpression.getValue().accept(exprVisitor, indent));
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 23c8273..92d4098 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
@@ -25,15 +25,29 @@
 
 import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.dataflow.std.base.RangeId;
 
 public class OrderedPartitionedProperty implements IPartitioningProperty {
 
     private List<OrderColumn> orderColumns;
     private INodeDomain domain;
+    private RangeId rangeId;
+    private RangePartitioningType rangeType;
+    private IRangeMap rangeMapHint;
 
-    public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain) {
+    public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, RangeId rangeId,
+            RangePartitioningType rangeType, IRangeMap rangeMapHint) {
         this.domain = domain;
         this.orderColumns = orderColumns;
+        this.rangeId = rangeId;
+        this.rangeType = rangeType;
+        this.rangeMapHint = rangeMapHint;
+    }
+
+    public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, RangeId rangeId) {
+        this(orderColumns, domain, rangeId, RangePartitioningType.PROJECT, null);
     }
 
     public List<OrderColumn> getOrderColumns() {
@@ -53,9 +67,13 @@
         return PartitioningType.ORDERED_PARTITIONED;
     }
 
+    public RangePartitioningType getRangePartitioningType() {
+        return rangeType;
+    }
+
     @Override
     public String toString() {
-        return getPartitioningType().toString() + orderColumns;
+        return getPartitioningType().toString() + " Column(s): " + orderColumns + " Range Type: " + rangeType;
     }
 
     @Override
@@ -63,7 +81,7 @@
             List<FunctionalDependency> fds) {
         List<OrderColumn> columns = PropertiesUtil.replaceOrderColumnsByEqClasses(orderColumns, equivalenceClasses);
         columns = PropertiesUtil.applyFDsToOrderColumns(columns, fds);
-        return new OrderedPartitionedProperty(columns, domain);
+        return new OrderedPartitionedProperty(columns, domain, rangeId);
     }
 
     @Override
@@ -73,6 +91,14 @@
         }
     }
 
+    public RangeId getRangeId() {
+        return rangeId;
+    }
+
+    public IRangeMap getRangeMapHint() {
+        return rangeMapHint;
+    }
+
     @Override
     public INodeDomain getNodeDomain() {
         return domain;
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 f2fed13..16b3f91 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
@@ -164,9 +164,13 @@
                         OrderedPartitionedProperty or = (OrderedPartitionedProperty) reqd;
                         OrderedPartitionedProperty od = (OrderedPartitionedProperty) dlvd;
                         if (mayExpandProperties) {
-                            return isPrefixOf(od.getOrderColumns().iterator(), or.getOrderColumns().iterator());
+                            return (isPrefixOf(od.getOrderColumns().iterator(), or.getOrderColumns().iterator())
+                                    && or.getRangePartitioningType().equals(od.getRangePartitioningType())
+                                    && or.getRangeId().equals(od.getRangeId()));
                         } else {
-                            return od.getOrderColumns().equals(or.getOrderColumns());
+                            return (or.getOrderColumns().equals(od.getOrderColumns())
+                                    && or.getRangePartitioningType().equals(od.getRangePartitioningType())
+                                    && or.getRangeId().equals(od.getRangeId()));
                         }
                     }
                     default: {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index f5ff8b4..85332ca 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -40,6 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -84,6 +85,8 @@
 
     public R visitReplicateOperator(ReplicateOperator op, T arg) throws AlgebricksException;
 
+    public R visitRangeForwardOperator(RangeForwardOperator op, T arg) throws AlgebricksException;
+
     public R visitSplitOperator(SplitOperator op, T arg) throws AlgebricksException;
 
     public R visitMaterializeOperator(MaterializeOperator op, T arg) throws AlgebricksException;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
index a1b3556..08a6edf 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AlgebricksOptimizationContext.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILogicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.dataflow.std.base.RangeId;
 
 /**
  * The Algebricks default implementation for IOptimizationContext.
@@ -51,6 +52,7 @@
 public class AlgebricksOptimizationContext implements IOptimizationContext {
 
     private int varCounter;
+    private int rangeIdCounter;
     private final IExpressionEvalSizeComputer expressionEvalSizeComputer;
     private final IMergeAggregationExpressionFactory mergeAggregationExpressionFactory;
     private final PhysicalOptimizationConfig physicalOptimizationConfig;
@@ -104,6 +106,7 @@
             IConflictingTypeResolver conflictingTypeResovler, PhysicalOptimizationConfig physicalOptimizationConfig,
             AlgebricksPartitionConstraint clusterLocations, LogicalOperatorPrettyPrintVisitor prettyPrintVisitor) {
         this.varCounter = varCounter;
+        this.rangeIdCounter = -1;
         this.expressionEvalSizeComputer = expressionEvalSizeComputer;
         this.mergeAggregationExpressionFactory = mergeAggregationExpressionFactory;
         this.expressionTypeComputer = expressionTypeComputer;
@@ -131,6 +134,13 @@
     }
 
     @Override
+    public RangeId newRangeId() {
+        rangeIdCounter++;
+        RangeId id = new RangeId(rangeIdCounter);
+        return id;
+    }
+
+    @Override
     public IMetadataProvider getMetadataProvider() {
         return metadataProvider;
     }
@@ -209,7 +219,7 @@
     @Override
     public List<LogicalVariable> findPrimaryKey(LogicalVariable recordVar) {
         FunctionalDependency fd = varToPrimaryKey.get(recordVar);
-        return fd == null ? null : new ArrayList<>(fd.getHead());
+        return fd == null ? new ArrayList<>() : new ArrayList<>(fd.getHead());
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 7340882..f28f5b8 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -30,6 +30,7 @@
     private static final String MAX_FRAMES_FOR_JOIN = "MAX_FRAMES_FOR_JOIN";
     private static final String FUDGE_FACTOR = "FUDGE_FACTOR";
     private static final String MAX_RECORDS_PER_FRAME = "MAX_RECORDS_PER_FRAME";
+    private static final String MAX_INTERVAL_DURATION = "MAX_INTERVAL_DURATION";
 
     private static final String DEFAULT_HASH_GROUP_TABLE_SIZE = "DEFAULT_HASH_GROUP_TABLE_SIZE";
     private static final String DEFAULT_EXTERNAL_GROUP_TABLE_SIZE = "DEFAULT_EXTERNAL_GROUP_TABLE_SIZE";
@@ -91,6 +92,14 @@
         setInt(MAX_FRAMES_FOR_JOIN, frameLimit);
     }
 
+    public int getMaxIntervalDuration() {
+        return getInt(MAX_INTERVAL_DURATION, 1000);
+    }
+
+    public void getMaxIntervalDuration(int duration) {
+        setInt(MAX_INTERVAL_DURATION, duration);
+    }
+
     public int getMaxFramesExternalGroupBy() {
         int frameSize = getFrameSize();
         return getInt(MAX_FRAMES_EXTERNAL_GROUP_BY, (int) (((long) 256 * MB) / frameSize));
diff --git a/hyracks-fullstack/algebricks/algebricks-data/pom.xml b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
index d95810a..b35694b 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-data/pom.xml
@@ -59,6 +59,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-common</artifactId>
+      <version>0.2.18-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
index bdeb018..f6b1796 100644
--- a/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
+++ b/hyracks-fullstack/algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
@@ -20,8 +20,13 @@
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
 
 public interface IBinaryComparatorFactoryProvider {
     public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending)
             throws AlgebricksException;
+
+    public IBinaryRangeComparatorFactory getRangeBinaryComparatorFactory(Object type, boolean ascending,
+            RangePartitioningType rangeType) throws AlgebricksException;
 }
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 159bcf5..bea6894 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
@@ -49,6 +49,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
@@ -60,6 +61,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
 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.RangeForwardPOperator;
 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;
@@ -89,7 +91,9 @@
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.dataflow.std.base.RangeId;
 
 public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
 
@@ -539,9 +543,12 @@
                         pop = new RandomMergeExchangePOperator();
                     } else {
                         if (op.getAnnotations().containsKey(OperatorAnnotations.USE_RANGE_CONNECTOR)) {
-                            IRangeMap rangeMap =
-                                    (IRangeMap) op.getAnnotations().get(OperatorAnnotations.USE_RANGE_CONNECTOR);
-                            pop = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap);
+                            RangeId rangeId = context.newRangeId();
+                            IRangeMap rangeMap = (IRangeMap) op.getAnnotations()
+                                    .get(OperatorAnnotations.USE_RANGE_CONNECTOR);
+                            addRangeForwardOperator(op.getInputs().get(i), rangeId, rangeMap, context);
+                            pop = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeId,
+                                    RangePartitioningType.PROJECT);
                         } else {
                             OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
                             sortColumns = ordCols.toArray(sortColumns);
@@ -573,8 +580,30 @@
                     break;
                 }
                 case ORDERED_PARTITIONED: {
-                    pop = new RangePartitionExchangePOperator(((OrderedPartitionedProperty) pp).getOrderColumns(),
-                            domain, null);
+                    OrderedPartitionedProperty opp = (OrderedPartitionedProperty) pp;
+                    List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+                    List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
+
+                    // The RangeForwardOperator should already be in the plan.
+
+                    boolean propWasSet = false;
+                    pop = null;
+                    if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+                        AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+                        Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
+                        List<FunctionalDependency> fds = context.getFDList(c);
+                        if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+                            List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals,
+                                    cldLocals);
+                            pop = new RangePartitionMergeExchangePOperator(orderColumns, domain, opp.getRangeId(),
+                                    opp.getRangePartitioningType());
+                            propWasSet = true;
+                        }
+                    }
+                    if (!propWasSet) {
+                        pop = new RangePartitionExchangePOperator(opp.getOrderColumns(), domain, opp.getRangeId(),
+                                opp.getRangePartitioningType());
+                    }
                     break;
                 }
                 case BROADCAST: {
@@ -607,6 +636,21 @@
         }
     }
 
+    private void addRangeForwardOperator(Mutable<ILogicalOperator> op, RangeId rangeId, IRangeMap rangeMap,
+            IOptimizationContext context) throws AlgebricksException {
+        RangeForwardOperator rfo = new RangeForwardOperator(rangeId, rangeMap);
+        RangeForwardPOperator rfpo = new RangeForwardPOperator(rangeId, rangeMap);
+        rfo.setPhysicalOperator(rfpo);
+        setNewOp(op, rfo, context);
+        rfo.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(rfo, context);
+        context.computeAndSetTypeEnvironmentForOperator(rfo);
+        if (AlgebricksConfig.DEBUG) {
+            AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added range forward " + rfo.getPhysicalOperator() + ".\n");
+            printOp((AbstractLogicalOperator) op.getValue());
+        }
+    }
+
     private boolean allAreOrderProps(List<ILocalStructuralProperty> cldLocals) {
         for (ILocalStructuralProperty lsp : cldLocals) {
             if (lsp.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 0c09fc0..21bc3d5 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -50,6 +50,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
@@ -69,6 +70,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.NestedTupleSourcePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangeForwardPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RunningAggregatePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.SinkPOperator;
@@ -233,6 +235,11 @@
                     op.setPhysicalOperator(new RunningAggregatePOperator());
                     break;
                 }
+                case RANGE_FORWARD: {
+                    RangeForwardOperator rfo = (RangeForwardOperator) op;
+                    op.setPhysicalOperator(new RangeForwardPOperator(rfo.getRangeId(), rfo.getRangeMap()));
+                    break;
+                }
                 case REPLICATE: {
                     op.setPhysicalOperator(new ReplicatePOperator());
                     break;
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index c5d7291..29e96f6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -49,6 +49,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.RangeForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
@@ -184,6 +185,11 @@
     }
 
     @Override
+    public ILogicalOperator visitRangeForwardOperator(RangeForwardOperator op, Void arg) throws AlgebricksException {
+        return visit(op);
+    }
+
+    @Override
     public ILogicalOperator visitMaterializeOperator(MaterializeOperator op, Void arg) throws AlgebricksException {
         return visit(op);
     }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryRangeComparatorFactory.java
similarity index 74%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryRangeComparatorFactory.java
index 500ac03..556156a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryRangeComparatorFactory.java
@@ -16,11 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.api.dataflow.value;
 
-drop dataverse test if exists;
-create dataverse test;
+import java.io.Serializable;
+
+public interface IBinaryRangeComparatorFactory extends Serializable {
+    public IBinaryComparator createMinBinaryComparator();
+    public IBinaryComparator createMaxBinaryComparator();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangeMap.java
similarity index 69%
rename from hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangeMap.java
index 5c5f34b..cd14434 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangeMap.java
@@ -16,12 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.dataflow.common.data.partition.range;
-
-import org.apache.hyracks.data.std.api.IPointable;
+package org.apache.hyracks.api.dataflow.value;
 
 public interface IRangeMap {
-    public IPointable getFieldSplit(int columnIndex, int splitIndex);
 
     public int getSplitCount();
 
@@ -32,4 +29,22 @@
     public int getLength(int columnIndex, int splitIndex);
 
     public int getTag(int columnIndex, int splitIndex);
+
+    // Min value functions
+    public byte[] getMinByteArray(int columnIndex);
+
+    public int getMinStartOffset(int columnIndex);
+
+    public int getMinLength(int columnIndex);
+
+    public int getMinTag(int columnIndex);
+
+    // Max value functions
+    public byte[] getMaxByteArray(int columnIndex);
+
+    public int getMaxStartOffset(int columnIndex);
+
+    public int getMaxLength(int columnIndex);
+
+    public int getMaxTag(int columnIndex);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangePartitionType.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangePartitionType.java
new file mode 100644
index 0000000..abf1495
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IRangePartitionType.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.dataflow.value;
+
+public interface IRangePartitionType {
+    public enum RangePartitioningType {
+        /**
+         * Partitioning is determined by finding the range partition where the first data point lies.
+         */
+        PROJECT,
+        /**
+         * Partitioning is determined by finding the range partition where the last data point lies.
+         */
+        PROJECT_END,
+        /**
+         * Partitioning is determined by finding all the range partitions where the data has a point.
+         */
+        SPLIT,
+        /**
+         * Partitioning is determined by finding all the range partitions where the data has a point
+         * or comes after the data point.
+         */
+        REPLICATE
+    }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
index 500ac03..fa51d5f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
@@ -16,11 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.api.dataflow.value;
 
-drop dataverse test if exists;
-create dataverse test;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.storage.IGrowableIntArray;
+
+public interface ITupleRangePartitionComputer {
+    public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, IGrowableIntArray map)
+            throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
similarity index 76%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
index 500ac03..9b3e607 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
@@ -16,11 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.api.dataflow.value;
 
-drop dataverse test if exists;
-create dataverse test;
+import java.io.Serializable;
+
+public interface ITupleRangePartitionComputerFactory extends Serializable {
+    public ITupleRangePartitionComputer createPartitioner(IRangeMap rangeMap);
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
similarity index 77%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
index 500ac03..9e804ba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
@@ -16,11 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.api.dataflow.value;
 
-drop dataverse test if exists;
-create dataverse test;
+import java.io.Serializable;
+
+public interface ITupleRangePartitionComputerFamily extends Serializable {
+    public ITupleRangePartitionComputer createPartitioner(int seed);
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
similarity index 82%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql
rename to hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
index f686b54..4e18b2b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins/overlap_bins.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
@@ -16,7 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/**
- * overlap_bins test case: test the overlap_bins
- * Expected result: success
- **/
+package org.apache.hyracks.api.storage;
+
+public interface IGrowableIntArray {
+
+    int size();
+
+    void add(int i);
+
+    int get(int i);
+
+    void clear();
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeAscComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeAscComparatorFactory.java
new file mode 100644
index 0000000..0bcff17
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeAscComparatorFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.data.std.accessors;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.data.std.api.IComparable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IPointableFactory;
+
+public class PointableBinaryRangeAscComparatorFactory implements IBinaryRangeComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    protected final IPointableFactory pf;
+
+    public PointableBinaryRangeAscComparatorFactory(IPointableFactory pf) {
+        this.pf = pf;
+    }
+
+    public static PointableBinaryRangeAscComparatorFactory of(IPointableFactory pf) {
+        return new PointableBinaryRangeAscComparatorFactory(pf);
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        final IPointable p = pf.createPointable();
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                if (l1 == 0 && l2 != 0)
+                    return -1;
+                if (l1 != 0 && l2 == 0)
+                    return 1;
+                p.set(b1, s1, l1);
+                return ((IComparable) p).compareTo(b2, s2, l2);
+            }
+        };
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return createMinBinaryComparator();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeDescComparatorFactory.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeDescComparatorFactory.java
new file mode 100644
index 0000000..01d3043
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeDescComparatorFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.data.std.accessors;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.data.std.api.IComparable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IPointableFactory;
+
+public class PointableBinaryRangeDescComparatorFactory implements IBinaryRangeComparatorFactory {
+    private static final long serialVersionUID = 1L;
+
+    protected final IPointableFactory pf;
+
+    public PointableBinaryRangeDescComparatorFactory(IPointableFactory pf) {
+        this.pf = pf;
+    }
+
+    public static PointableBinaryRangeDescComparatorFactory of(IPointableFactory pf) {
+        return new PointableBinaryRangeDescComparatorFactory(pf);
+    }
+
+    @Override
+    public IBinaryComparator createMinBinaryComparator() {
+        final IPointable p = pf.createPointable();
+        return new IBinaryComparator() {
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                if (l1 == 0 && l2 != 0)
+                    return -1;
+                if (l1 != 0 && l2 == 0)
+                    return 1;
+                p.set(b1, s1, l1);
+                return -((IComparable) p).compareTo(b2, s2, l2);
+            }
+        };
+    }
+
+    @Override
+    public IBinaryComparator createMaxBinaryComparator() {
+        return createMinBinaryComparator();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
index d99e2f2..1c702be 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTupleAccessor.java
@@ -35,7 +35,8 @@
 /**
  * FrameTupleCursor is used to navigate over tuples in a Frame. A frame is
  * formatted with tuple data concatenated starting at offset 1, one tuple after
- * another. The first byte is used to notify how big the frame is, so the maximum frame size is 255 * initialFrameSetting.
+ * another. The first byte is used to notify how big the frame is,
+ * so the maximum frame size is 255 * initialFrameSetting.
  * Offset FS - 4 holds an int indicating the number of tuples (N) in
  * the frame. FS - ((i + 1) * 4) for i from 0 to N - 1 holds an int indicating
  * the offset of the (i + 1)^th tuple. Every tuple is organized as a sequence of
@@ -184,6 +185,7 @@
             prettyPrint(i, bbis, dis, sb, recordFields);
         }
         System.err.println(sb.toString());
+        bbis.close();
     }
 
     public void prettyPrint(int tIdx, int[] recordFields) throws IOException {
@@ -192,6 +194,7 @@
         StringBuilder sb = new StringBuilder();
         prettyPrint(tIdx, bbis, dis, sb, recordFields);
         System.err.println(sb.toString());
+        bbis.close();
     }
 
     public void prettyPrint(ITupleReference tuple, int fieldsIdx, int descIdx) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
index f4da9bf..162d3b4 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
@@ -20,65 +20,142 @@
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.storage.IGrowableIntArray;
 
-public class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
+public class FieldRangePartitionComputerFactory implements ITupleRangePartitionComputerFactory {
     private static final long serialVersionUID = 1L;
     private final int[] rangeFields;
-    private IRangeMap rangeMap;
-    private IBinaryComparatorFactory[] comparatorFactories;
+    private IBinaryRangeComparatorFactory[] comparatorFactories;
+    private RangePartitioningType rangeType;
 
-    public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
-            IRangeMap rangeMap) {
+    public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryRangeComparatorFactory[] comparatorFactories,
+            RangePartitioningType rangeType) {
         this.rangeFields = rangeFields;
         this.comparatorFactories = comparatorFactories;
-        this.rangeMap = rangeMap;
+        this.rangeType = rangeType;
     }
 
-    @Override
-    public ITuplePartitionComputer createPartitioner() {
-        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+    public ITupleRangePartitionComputer createPartitioner(IRangeMap rangeMap) {
+        final IBinaryComparator[] minComparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
-            comparators[i] = comparatorFactories[i].createBinaryComparator();
+            minComparators[i] = comparatorFactories[i].createMinBinaryComparator();
         }
-        return new ITuplePartitionComputer() {
-            @Override
-            /**
-             * Determine the range partition.
-             */
-            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+        final IBinaryComparator[] maxComparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            maxComparators[i] = comparatorFactories[i].createMaxBinaryComparator();
+        }
+        return new ITupleRangePartitionComputer() {
+            private int partionCount;
+            private double rangesPerPart = 1;
+
+            public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, IGrowableIntArray map)
+                    throws HyracksDataException {
                 if (nParts == 1) {
-                    return 0;
+                    map.add(0);
+                    return;
                 }
-                int slotIndex = getRangePartition(accessor, tIndex);
                 // Map range partition to node partitions.
-                double rangesPerPart = 1;
-                if (rangeMap.getSplitCount() + 1 > nParts) {
-                    rangesPerPart = ((double) rangeMap.getSplitCount() + 1) / nParts;
+                if (partionCount != nParts) {
+                    partionCount = nParts;
+                    if (rangeMap.getSplitCount() + 1 > nParts) {
+                        rangesPerPart = ((double) rangeMap.getSplitCount() + 1) / nParts;
+                    }
                 }
-                return (int) Math.floor(slotIndex / rangesPerPart);
+                getRangePartitions(accessor, tIndex, map);
             }
 
             /*
-             * Determine the range partition.
+             * Determine the range partitions.
              */
-            public int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
-                int slotIndex = 0;
-                for (int i = 0; i < rangeMap.getSplitCount(); ++i) {
-                    int c = compareSlotAndFields(accessor, tIndex, i);
-                    if (c < 0) {
-                        return slotIndex;
+            private void getRangePartitions(IFrameTupleAccessor accessor, int tIndex, IGrowableIntArray map)
+                    throws HyracksDataException {
+                switch (rangeType) {
+                    case PROJECT: {
+                        int minPartition = getPartitionMap(
+                                binarySearchRangePartition(accessor, tIndex, minComparators));
+                        addPartition(minPartition, map);
+                        break;
                     }
-                    slotIndex++;
+                    case PROJECT_END: {
+                        int maxPartition = getPartitionMap(
+                                binarySearchRangePartition(accessor, tIndex, maxComparators));
+                        addPartition(maxPartition, map);
+                        break;
+                    }
+                    case REPLICATE: {
+                        int minPartition = getPartitionMap(
+                                binarySearchRangePartition(accessor, tIndex, minComparators));
+                        int maxPartition = getPartitionMap(rangeMap.getSplitCount() + 1);
+                        for (int pid = minPartition; pid < maxPartition; ++pid) {
+                            addPartition(pid, map);
+                        }
+                        break;
+                    }
+                    case SPLIT: {
+                        int minPartition = getPartitionMap(
+                                binarySearchRangePartition(accessor, tIndex, minComparators));
+                        int maxPartition = getPartitionMap(
+                                binarySearchRangePartition(accessor, tIndex, maxComparators));
+                        for (int pid = minPartition; pid <= maxPartition; ++pid) {
+                            addPartition(pid, map);
+                        }
+                        break;
+                    }
+                    default:
                 }
-                return slotIndex;
             }
 
-            public int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int fieldIndex)
-                    throws HyracksDataException {
+            private void addPartition(int partition, IGrowableIntArray map) {
+                if (!hasPartition(partition, map)) {
+                    map.add(partition);
+                }
+            }
+
+            private int getPartitionMap(int partition) {
+                return (int) Math.floor(partition / rangesPerPart);
+            }
+
+            private boolean hasPartition(int pid, IGrowableIntArray map) {
+                for (int i = 0; i < map.size(); ++i) {
+                    if (map.get(i) == pid) {
+                        return true;
+                    }
+                }
+                return false;
+            }
+
+            /*
+             * Return first match or suggested index.
+             */
+            private int binarySearchRangePartition(IFrameTupleAccessor accessor, int tIndex,
+                    IBinaryComparator[] comparators) throws HyracksDataException {
+                int searchIndex = 0;
+                int left = 0;
+                int right = rangeMap.getSplitCount() - 1;
+                int cmp;
+                while (left <= right) {
+                    searchIndex = (left + right) / 2;
+                    cmp = compareSlotAndFields(accessor, tIndex, searchIndex, comparators);
+                    if (cmp > 0) {
+                        left = searchIndex + 1;
+                        searchIndex += 1;
+                    } else if (cmp < 0) {
+                        right = searchIndex - 1;
+                    } else {
+                        return searchIndex + 1;
+                    }
+                }
+                return searchIndex;
+            }
+
+            private int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int mapIndex,
+                    IBinaryComparator[] comparators) throws HyracksDataException {
                 int c = 0;
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int slotLength = accessor.getFieldSlotsLength();
@@ -86,9 +163,9 @@
                     int fIdx = rangeFields[f];
                     int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
                     int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
-                    c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd
-                            - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f),
-                            rangeMap.getLength(fieldIndex, f));
+                    c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+                            fEnd - fStart, rangeMap.getByteArray(f, mapIndex), rangeMap.getStartOffset(f, mapIndex),
+                            rangeMap.getLength(f, mapIndex));
                     if (c != 0) {
                         return c;
                     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
index 98acbc0..c15d39a 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
@@ -20,14 +20,17 @@
 
 import java.io.Serializable;
 
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
 
 /**
  * The range map stores the field split values in an byte array.
- * The first split value for each field followed by the second split value for each field, etc.
+ * The first and last split values for each column represent the min and max values (not actually split values).
+ * <br />
+ * Example for columns A and B with three split values.
+ * {min A, min B, split 1 A, split 1 B, split 2 A, split 2 B, split 3 A, split 3 B, max A, max B}
  */
 public class RangeMap implements IRangeMap, Serializable {
+    private static final long serialVersionUID = 1L;
     private final int fields;
     private final byte[] bytes;
     private final int[] offsets;
@@ -39,16 +42,8 @@
     }
 
     @Override
-    public IPointable getFieldSplit(int columnIndex, int splitIndex) {
-        IPointable p = VoidPointable.FACTORY.createPointable();
-        int index = getFieldIndex(columnIndex, splitIndex);
-        p.set(bytes, getFieldStart(index), getFieldLength(index));
-        return p;
-    }
-
-    @Override
     public int getSplitCount() {
-        return offsets.length / fields;
+        return offsets.length / fields - 2;
     }
 
     @Override
@@ -58,21 +53,21 @@
 
     @Override
     public int getTag(int columnIndex, int splitIndex) {
-        return getFieldTag(getFieldIndex(columnIndex, splitIndex));
+        return getFieldTag(getFieldIndex(columnIndex, splitIndex + 1));
     }
 
     @Override
     public int getStartOffset(int columnIndex, int splitIndex) {
-        return getFieldStart(getFieldIndex(columnIndex, splitIndex));
+        return getFieldStart(getFieldIndex(columnIndex, splitIndex + 1));
     }
 
     @Override
     public int getLength(int columnIndex, int splitIndex) {
-        return getFieldLength(getFieldIndex(columnIndex, splitIndex));
+        return getFieldLength(getFieldIndex(columnIndex, splitIndex + 1));
     }
 
     private int getFieldIndex(int columnIndex, int splitIndex) {
-        return splitIndex * fields + columnIndex;
+        return columnIndex + splitIndex * fields;
     }
 
     private int getFieldTag(int index) {
@@ -95,4 +90,52 @@
         return length;
     }
 
+    @Override
+    public byte[] getMinByteArray(int columnIndex) {
+        return bytes;
+    }
+
+    @Override
+    public int getMinStartOffset(int columnIndex) {
+        return getFieldStart(getFieldIndex(columnIndex, getMinIndex()));
+    }
+
+    @Override
+    public int getMinLength(int columnIndex) {
+        return getFieldLength(getFieldIndex(columnIndex, getMinIndex()));
+    }
+
+    @Override
+    public int getMinTag(int columnIndex) {
+        return getFieldTag(getFieldIndex(columnIndex, getMinIndex()));
+    }
+
+    @Override
+    public byte[] getMaxByteArray(int columnIndex) {
+        return bytes;
+    }
+
+    @Override
+    public int getMaxStartOffset(int columnIndex) {
+        return getFieldStart(getFieldIndex(columnIndex, getMaxIndex()));
+    }
+
+    @Override
+    public int getMaxLength(int columnIndex) {
+        return getFieldLength(getFieldIndex(columnIndex, getMaxIndex()));
+    }
+
+    @Override
+    public int getMaxTag(int columnIndex) {
+        return getFieldTag(getFieldIndex(columnIndex, getMaxIndex()));
+    }
+
+    private int getMaxIndex() {
+        return offsets.length / fields - 1;
+    }
+
+    private int getMinIndex() {
+        return 0;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
index fb160f0..06de054 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileReader.java
@@ -32,6 +32,7 @@
     private IFileHandle handle;
     private final IIOManager ioManager;
     private final long size;
+    private long readPreviousPtr;
     private long readPtr;
     private boolean deleteAfterClose;
 
@@ -46,6 +47,7 @@
     public void open() throws HyracksDataException {
         handle = ioManager.open(file, IIOManager.FileReadWriteMode.READ_ONLY, null);
         readPtr = 0;
+        readPreviousPtr = 0;
     }
 
     @Override
@@ -53,6 +55,7 @@
         if (readPtr >= size) {
             return false;
         }
+        readPreviousPtr = readPtr;
         frame.reset();
 
         int readLength = ioManager.syncRead(handle, readPtr, frame.getBuffer());
@@ -77,6 +80,16 @@
         return true;
     }
 
+    public void reset() throws HyracksDataException {
+        readPtr = 0;
+        readPreviousPtr = readPtr;
+    }
+
+    public void reset(long pointer) throws HyracksDataException {
+        readPtr = pointer;
+        readPreviousPtr = readPtr;
+    }
+
     @Override
     public void close() throws HyracksDataException {
         ioManager.close(handle);
@@ -88,4 +101,12 @@
     public long getFileSize() {
         return size;
     }
+
+    public long getReadPointer() {
+        return readPreviousPtr;
+    }
+
+    public long getReadPointerCurrent() {
+        return readPtr;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
index b610220..29d41ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/pom.xml
@@ -77,6 +77,13 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-storage-common</artifactId>
+      <version>0.2.18-SNAPSHOT</version>
+      <type>jar</type>
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/RangeId.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/RangeId.java
new file mode 100644
index 0000000..e1b7b12
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/RangeId.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.dataflow.std.base;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Represents a range id in a logical plan.
+ */
+public final class RangeId implements Serializable {
+    private static final long serialVersionUID = 1L;
+    private final int id;
+    private int partition = -1;
+
+    public RangeId(int id, int partition) {
+        this.id = id;
+        this.partition = partition;
+    }
+
+    public RangeId(int id, IHyracksTaskContext ctx) {
+        this.id = id;
+        this.partition = ctx.getTaskAttemptId().getTaskId().getPartition();
+    }
+
+    public RangeId(int id) {
+        this.id = id;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    public int getPartition() {
+        return partition;
+    }
+
+    public void setPartition(int partition) {
+        this.partition = partition;
+    }
+
+    @Override
+    public String toString() {
+        return "RangeId(" + id + (partition >= 0 ? "," + partition : "") + ")";
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof RangeId)) {
+            return false;
+        } else {
+            return id == ((RangeId) obj).getId() && partition == ((RangeId) obj).getPartition();
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return id;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java
new file mode 100644
index 0000000..4db1ab7
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.dataflow.std.buffermanager;
+
+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;
+
+    abstract IFrameTupleAccessor getInnerAccessor();
+
+    abstract void resetInnerAccessor(int frameId);
+
+    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(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;
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
index 1118bf3..5741071 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IFrameBufferManager.java
@@ -36,7 +36,8 @@
 
     /**
      * @param frameIndex
-     * @param bufferInfo the given object need to be reset
+     * @param bufferInfo
+     *            the given object need to be reset
      * @return the filled bufferInfo to facilitate the chain access
      */
     BufferInfo getFrame(int frameIndex, BufferInfo bufferInfo);
@@ -49,11 +50,30 @@
     /**
      * Writes the whole frame into the buffer.
      *
-     * @param frame source frame
+     * @param frame
+     *            source frame
      * @return the id of the inserted frame. return -1 if it failed to insert
      */
     int insertFrame(ByteBuffer frame) throws HyracksDataException;
 
+    /**
+     * Removes the frame from the buffer manager
+     *
+     * @param frameIndex
+     */
+    void removeFrame(int frameIndex);
+
     void close();
 
+    /**
+     * Create a iterator for frames.
+     *
+     * Allows the reuse of frame ids.
+     */
+    int next();
+
+    boolean exists();
+
+    void resetIterator();
+
 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedDeletableTupleBufferManager.java
similarity index 65%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedDeletableTupleBufferManager.java
index 500ac03..52a0918 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedDeletableTupleBufferManager.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
 
-drop dataverse test if exists;
-create dataverse test;
+package org.apache.hyracks.dataflow.std.buffermanager;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface IPartitionedDeletableTupleBufferManager extends IPartitionedTupleBufferManager {
+
+    void deleteTuple(int partition, TuplePointer tuplePointer) throws HyracksDataException;
+
+    void printStats(String string);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
index a0a9ab0..c0c96b3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
@@ -31,6 +31,8 @@
 
     int getNumTuples(int partition);
 
+    int getNumFrames(int partition);
+
     int getPhysicalSize(int partition);
 
     /**
@@ -59,7 +61,7 @@
     /**
      * Insert tuple {@code tupleId} from the {@code tupleAccessor} into the given partition.
      * The returned handle is written into the tuplepointer
-     * 
+     *
      * @param partition
      *            the id of the partition to insert the tuple
      * @param tupleAccessor
@@ -76,7 +78,7 @@
 
     /**
      * Reset to the initial states. The previous allocated resources won't be released in order to be used in the next round.
-     * 
+     *
      * @throws HyracksDataException
      */
     void reset() throws HyracksDataException;
@@ -88,21 +90,23 @@
 
     ITuplePointerAccessor getTuplePointerAccessor(RecordDescriptor recordDescriptor);
 
+    ITupleAccessor getTupleAccessor(RecordDescriptor recordDescriptor);
+
     /**
      * Flush the particular partition {@code pid} to {@code writer}.
      * This partition will not be cleared.
      * Currently it is used by Join where we flush the inner partition to the join (as a frameWriter),
      * but we will still keep the inner for the next outer partition.
-     * 
+     *
      * @param pid
      * @param writer
      * @throws HyracksDataException
      */
-    void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException;
+    void flushPartition(int partition, IFrameWriter writer) throws HyracksDataException;
 
     /**
      * Clear the memory occupation of the particular partition.
-     * 
+     *
      * @param partition
      * @throws HyracksDataException
      */
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java
new file mode 100644
index 0000000..78be026
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.dataflow.std.buffermanager;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+// TODO determine correct interface.
+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);
+
+    @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/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
index 94fc643..14ae6d4 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
@@ -44,4 +44,6 @@
     void close() throws HyracksDataException;
 
     ITuplePointerAccessor createTuplePointerAccessor();
+
+    ITupleAccessor createTupleAccessor();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java
new file mode 100644
index 0000000..84857c0
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.buffermanager;
+
+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;
+    }
+
+    @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/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionDeletableTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionDeletableTupleBufferManager.java
new file mode 100644
index 0000000..6802e19
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionDeletableTupleBufferManager.java
@@ -0,0 +1,282 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.dataflow.std.buffermanager;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksFrameMgrContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+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;
+
+/**
+ * This buffer manager will divide the buffers into given number of partitions.
+ * The cleared partition (spilled one in the caller side) can only get no more than one frame.
+ */
+public class VPartitionDeletableTupleBufferManager extends VPartitionTupleBufferManager
+        implements IPartitionedDeletableTupleBufferManager {
+
+    private final int[] minFreeSpace;
+    private final IAppendDeletableFrameTupleAccessor[] accessor;
+    private final IFrameFreeSlotPolicy[] policy;
+
+    public VPartitionDeletableTupleBufferManager(IHyracksFrameMgrContext ctx, IPartitionedMemoryConstrain constrain,
+            int partitions, int frameLimitInBytes, RecordDescriptor[] recordDescriptors) throws HyracksDataException {
+        super(ctx, constrain, partitions, frameLimitInBytes);
+        int maxFrames = framePool.getMemoryBudgetBytes() / framePool.getMinFrameSize();
+        this.policy = new FrameFreeSlotLastFit[partitions];
+        this.accessor = new DeletableFrameTupleAppender[partitions];
+        this.minFreeSpace = new int[partitions];
+        int i = 0;
+        for (RecordDescriptor rd : recordDescriptors) {
+            this.accessor[i] = new DeletableFrameTupleAppender(rd);
+            this.minFreeSpace[i] = calculateMinFreeSpace(rd);
+            this.policy[i] = new FrameFreeSlotLastFit(maxFrames);
+            ++i;
+        }
+    }
+
+    @Override
+    public boolean insertTuple(int partition, IFrameTupleAccessor fta, int idx, TuplePointer tuplePointer)
+            throws HyracksDataException {
+        int requiredFreeSpace = calculatePhysicalSpace(fta, idx);
+        int frameId = findAvailableFrame(partition, requiredFreeSpace);
+        if (frameId < 0) {
+            if (canBeInsertedAfterCleanUpFragmentation(partition, requiredFreeSpace)) {
+                reOrganizeFrames(partition);
+                frameId = findAvailableFrame(partition, requiredFreeSpace);
+            } else {
+                return false;
+            }
+        }
+        assert frameId >= 0;
+        partitionArray[partition].getFrame(frameId, tempInfo);
+        accessor[partition].reset(tempInfo.getBuffer());
+        assert accessor[partition].getContiguousFreeSpace() >= requiredFreeSpace;
+        int tid = accessor[partition].append(fta, idx);
+        assert tid >= 0;
+        if (accessor[partition].getContiguousFreeSpace() > minFreeSpace[partition]) {
+            policy[partition].pushNewFrame(frameId, accessor[partition].getContiguousFreeSpace());
+        }
+        tuplePointer.reset(makeGroupFrameId(partition, frameId), tid);
+        numTuples[partition]++;
+        return true;
+    }
+
+    @Override
+    public void clearPartition(int partitionId) throws HyracksDataException {
+        IFrameBufferManager partition = partitionArray[partitionId];
+        if (partition != null) {
+            partition.resetIterator();
+            int i = partition.next();
+            while (partition.exists()) {
+                accessor[partitionId].clear(partition.getFrame(i, tempInfo).getBuffer());
+                i = partition.next();
+            }
+        }
+        policy[partitionId].reset();
+        super.clearPartition(partitionId);
+    }
+
+    private void reOrganizeFrames(int partition) {
+        System.err.printf("reOrganizeFrames -- %d:[", partition);
+        policy[partition].reset();
+        partitionArray[partition].resetIterator();
+        int f = partitionArray[partition].next();
+        while (partitionArray[partition].exists()) {
+            partitionArray[partition].getFrame(f, tempInfo);
+            accessor[partition].reset(tempInfo.getBuffer());
+            accessor[partition].reOrganizeBuffer();
+            if (accessor[partition].getTupleCount() == 0) {
+                partitionArray[partition].removeFrame(f);
+                framePool.deAllocateBuffer(tempInfo.getBuffer());
+            } else {
+                policy[partition].pushNewFrame(f, accessor[partition].getContiguousFreeSpace());
+                accessor[partition].printStats(System.err);
+            }
+            f = partitionArray[partition].next();
+        }
+        System.err.println("] ");
+    }
+
+    private boolean canBeInsertedAfterCleanUpFragmentation(int partition, int requiredFreeSpace) {
+        partitionArray[partition].resetIterator();
+        int i = partitionArray[partition].next();
+        while (partitionArray[partition].exists()) {
+            partitionArray[partition].getFrame(i, tempInfo);
+            accessor[partition].reset(tempInfo.getBuffer());
+            if (accessor[partition].getTotalFreeSpace() >= requiredFreeSpace) {
+                return true;
+            }
+            i = partitionArray[partition].next();
+        }
+        return false;
+    }
+
+    private int findAvailableFrame(int partition, int requiredFreeSpace) throws HyracksDataException {
+        int frameId = policy[partition].popBestFit(requiredFreeSpace);
+        if (frameId >= 0) {
+            return frameId;
+        }
+        if (partitionArray[partition] == null || partitionArray[partition].getNumFrames() == 0) {
+            partitionArray[partition] = new PartitionFrameBufferManager();
+        }
+        return createNewBuffer(partition, requiredFreeSpace);
+    }
+
+    private static int calculatePhysicalSpace(IFrameTupleAccessor fta, int idx) {
+        // 4 bytes to store the offset
+        return 4 + fta.getTupleLength(idx);
+    }
+
+    private static int calculateMinFreeSpace(RecordDescriptor recordDescriptor) {
+        // + 4 for the tuple offset
+        return recordDescriptor.getFieldCount() * 4 + 4;
+    }
+
+    public void printStats(String message) {
+        System.err.print(String.format("%1$-" + 15 + "s", message) + " --");
+
+        for (int p = 0; p < partitionArray.length; ++p) {
+            System.err.printf("%d:[", p);
+            IFrameBufferManager partition = partitionArray[p];
+            if (partition != null) {
+                partitionArray[p].resetIterator();
+                int f = partitionArray[p].next();
+                while (partitionArray[p].exists()) {
+                    partitionArray[p].getFrame(f, tempInfo);
+                    accessor[p].reset(tempInfo.getBuffer());
+                    accessor[p].printStats(System.err);
+                    f = partitionArray[p].next();
+                }
+            }
+            System.err.printf("] ");
+        }
+        System.err.println();
+    }
+
+    @Override
+    public void deleteTuple(int partition, TuplePointer tuplePointer) throws HyracksDataException {
+        partitionArray[parsePartitionId(tuplePointer.getFrameIndex())]
+                .getFrame(parseFrameIdInPartition(tuplePointer.getFrameIndex()), tempInfo);
+        accessor[partition].reset(tempInfo.getBuffer());
+        accessor[partition].delete(tuplePointer.getTupleIndex());
+        numTuples[partition]--;
+    }
+
+    @Override
+    public ITuplePointerAccessor getTuplePointerAccessor(final RecordDescriptor recordDescriptor) {
+        return new AbstractTuplePointerAccessor() {
+            private IAppendDeletableFrameTupleAccessor innerAccessor = new DeletableFrameTupleAppender(
+                    recordDescriptor);
+
+            @Override
+            IFrameTupleAccessor getInnerAccessor() {
+                return innerAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(TuplePointer tuplePointer) {
+                partitionArray[parsePartitionId(tuplePointer.getFrameIndex())]
+                        .getFrame(parseFrameIdInPartition(tuplePointer.getFrameIndex()), tempInfo);
+                innerAccessor.reset(tempInfo.getBuffer());
+            }
+        };
+    }
+
+    @Override
+    public ITupleAccessor getTupleAccessor(final RecordDescriptor recordDescriptor) {
+        return new AbstractTupleAccessor() {
+            private DeletableFrameTupleAppender innerAccessor = new DeletableFrameTupleAppender(recordDescriptor);
+
+            @Override
+            IFrameTupleAccessor getInnerAccessor() {
+                return innerAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(int frameIndex) {
+                partitionArray[parsePartitionId(frameIndex)].getFrame(parseFrameIdInPartition(frameIndex), tempInfo);
+                innerAccessor.reset(tempInfo.getBuffer());
+            }
+
+            @Override
+            int getFrameCount() {
+                return partitionArray.length;
+            }
+
+            @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/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
index 1ed34f6..a094061 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
@@ -46,13 +46,13 @@
         }
     };
 
-    private IDeallocatableFramePool framePool;
-    private IFrameBufferManager[] partitionArray;
-    private int[] numTuples;
+    protected IDeallocatableFramePool framePool;
+    protected IFrameBufferManager[] partitionArray;
+    protected int[] numTuples;
     private final FixedSizeFrame appendFrame;
     private final FixedSizeFrameTupleAppender appender;
-    private BufferInfo tempInfo;
-    private final IPartitionedMemoryConstrain constrain;
+    protected BufferInfo tempInfo;
+    protected final IPartitionedMemoryConstrain constrain;
 
     public VPartitionTupleBufferManager(IHyracksFrameMgrContext ctx, IPartitionedMemoryConstrain constrain,
             int partitions, int frameLimitInBytes) throws HyracksDataException {
@@ -90,6 +90,11 @@
     }
 
     @Override
+    public int getNumFrames(int partition) {
+        return partitionArray[partition].getNumFrames();
+    }
+
+    @Override
     public int getPhysicalSize(int partitionId) {
         int size = 0;
         IFrameBufferManager partition = partitionArray[partitionId];
@@ -105,8 +110,11 @@
     public void clearPartition(int partitionId) throws HyracksDataException {
         IFrameBufferManager partition = partitionArray[partitionId];
         if (partition != null) {
-            for (int i = 0; i < partition.getNumFrames(); ++i) {
+            partition.resetIterator();
+            int i = partition.next();
+            while (partition.exists()) {
                 framePool.deAllocateBuffer(partition.getFrame(i, tempInfo).getBuffer());
+                i = partition.next();
             }
             partition.reset();
         }
@@ -146,26 +154,26 @@
                 tupleAccessor.getTupleStartOffset(tupleId), tupleAccessor.getTupleLength(tupleId), pointer);
     }
 
-    private static int calculateActualSize(int[] fieldEndOffsets, int size) {
+    protected static int calculateActualSize(int[] fieldEndOffsets, int size) {
         if (fieldEndOffsets != null) {
             return FrameHelper.calcRequiredSpace(fieldEndOffsets.length, size);
         }
         return FrameHelper.calcRequiredSpace(0, size);
     }
 
-    private int makeGroupFrameId(int partition, int fid) {
+    protected int makeGroupFrameId(int partition, int fid) {
         return fid * getNumPartitions() + partition;
     }
 
-    private int parsePartitionId(int externalFrameId) {
+    protected int parsePartitionId(int externalFrameId) {
         return externalFrameId % getNumPartitions();
     }
 
-    private int parseFrameIdInPartition(int externalFrameId) {
+    protected int parseFrameIdInPartition(int externalFrameId) {
         return externalFrameId / getNumPartitions();
     }
 
-    private int createNewBuffer(int partition, int size) throws HyracksDataException {
+    protected int createNewBuffer(int partition, int size) throws HyracksDataException {
         ByteBuffer newBuffer = requestNewBufferFromPool(size);
         if (newBuffer == null) {
             return -1;
@@ -180,8 +188,8 @@
         return framePool.allocateFrame(frameSize);
     }
 
-    private int appendTupleToBuffer(BufferInfo bufferInfo, int[] fieldEndOffsets, byte[] byteArray, int start, int size)
-            throws HyracksDataException {
+    protected int appendTupleToBuffer(BufferInfo bufferInfo, int[] fieldEndOffsets, byte[] byteArray, int start,
+            int size) throws HyracksDataException {
         assert (bufferInfo.getStartOffset() == 0) : "Haven't supported yet in FrameTupleAppender";
         if (bufferInfo.getBuffer() != appendFrame.getBuffer()) {
             appendFrame.reset(bufferInfo.getBuffer());
@@ -200,7 +208,7 @@
         return -1;
     }
 
-    private int getLastBufferOrCreateNewIfNotExist(int partition, int actualSize) throws HyracksDataException {
+    protected int getLastBufferOrCreateNewIfNotExist(int partition, int actualSize) throws HyracksDataException {
         if (partitionArray[partition] == null || partitionArray[partition].getNumFrames() == 0) {
             partitionArray[partition] = new PartitionFrameBufferManager();
             return createNewBuffer(partition, actualSize);
@@ -214,13 +222,15 @@
         Arrays.fill(partitionArray, null);
     }
 
-    private static class PartitionFrameBufferManager implements IFrameBufferManager {
+    static class PartitionFrameBufferManager implements IFrameBufferManager {
 
+        int size = 0;
         ArrayList<ByteBuffer> buffers = new ArrayList<>();
 
         @Override
         public void reset() throws HyracksDataException {
             buffers.clear();
+            size = 0;
         }
 
         @Override
@@ -231,13 +241,35 @@
 
         @Override
         public int getNumFrames() {
-            return buffers.size();
+            return size;
         }
 
         @Override
         public int insertFrame(ByteBuffer frame) throws HyracksDataException {
-            buffers.add(frame);
-            return buffers.size() - 1;
+            int index = -1;
+            if (buffers.size() == size) {
+                buffers.add(frame);
+                index = buffers.size() - 1;
+            } else {
+                for (int i = 0; i < buffers.size(); ++i) {
+                    if (buffers.get(i) == null) {
+                        buffers.set(i, frame);
+                        index = i;
+                        break;
+                    }
+                }
+            }
+            if (index == -1) {
+                throw new HyracksDataException("Did not insert frame.");
+            }
+            size++;
+            return index;
+        }
+
+        @Override
+        public void removeFrame(int frameIndex) {
+            buffers.set(frameIndex, null);
+            size--;
         }
 
         @Override
@@ -245,6 +277,28 @@
             buffers = null;
         }
 
+        int iterator = -1;
+
+        @Override
+        public int next() {
+            while (++iterator < buffers.size()) {
+                if (buffers.get(iterator) != null) {
+                    break;
+                }
+            }
+            return iterator;
+        }
+
+        @Override
+        public boolean exists() {
+            return iterator < buffers.size() && buffers.get(iterator) != null;
+        }
+
+        @Override
+        public void resetIterator() {
+            iterator = -1;
+        }
+
     }
 
     @Override
@@ -267,6 +321,29 @@
     }
 
     @Override
+    public ITupleAccessor getTupleAccessor(final RecordDescriptor recordDescriptor) {
+        return new AbstractTupleAccessor() {
+            FrameTupleAccessor innerAccessor = new FrameTupleAccessor(recordDescriptor);
+
+            @Override
+            IFrameTupleAccessor getInnerAccessor() {
+                return innerAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(int frameIndex) {
+                partitionArray[parsePartitionId(frameIndex)].getFrame(parseFrameIdInPartition(frameIndex), tempInfo);
+                innerAccessor.reset(tempInfo.getBuffer(), tempInfo.getStartOffset(), tempInfo.getLength());
+            }
+
+            @Override
+            int getFrameCount() {
+                return partitionArray.length;
+            }
+        };
+    }
+
+    @Override
     public void flushPartition(int pid, IFrameWriter writer) throws HyracksDataException {
         IFrameBufferManager partition = partitionArray[pid];
         if (partition != null && getNumTuples(pid) > 0) {
@@ -280,4 +357,11 @@
 
     }
 
+    public IFrameBufferManager getPartitionFrameBufferManager(int partition) {
+        if (partitionArray[partition] == null || partitionArray[partition].getNumFrames() == 0) {
+            return null;
+        }
+        return partitionArray[partition];
+    }
+
 }
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 4ed11e6..636559c 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
@@ -185,4 +185,80 @@
         };
     }
 
+    public ITupleAccessor createTupleAccessor() {
+        return new AbstractTupleAccessor() {
+            private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
+                    recordDescriptor);
+
+            @Override
+            IFrameTupleAccessor getInnerAccessor() {
+                return bufferAccessor;
+            }
+
+            @Override
+            void resetInnerAccessor(int frameIndex) {
+                bufferAccessor.reset(frames.get(frameIndex));
+            }
+
+            @Override
+            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/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
index 6604ba8..7c750d7 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableFrameMemoryManager.java
@@ -104,10 +104,33 @@
     }
 
     @Override
+    public void removeFrame(int frameIndex)  {
+        logicalFrameStartSizes.remove(frameIndex);
+    }
+
+    @Override
     public void close() {
         physicalFrameOffsets.clear();
         logicalFrameStartSizes.clear();
         freeSlotPolicy.reset();
         framePool.close();
     }
+
+    @Override
+    public int next() {
+        // TODO Auto-generated method stub
+        return 0;
+    }
+
+    @Override
+    public boolean exists() {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public void resetIterator() {
+        // TODO Auto-generated method stub
+
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
new file mode 100644
index 0000000..b9964a8
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.connectors;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+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.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public abstract class AbstractPartitionDataWriter implements IFrameWriter {
+    protected final int consumerPartitionCount;
+    protected final IFrameWriter[] pWriters;
+    protected final FrameTupleAppender[] appenders;
+    protected final FrameTupleAccessor tupleAccessor;
+    protected final IHyracksTaskContext ctx;
+    protected boolean allocatedFrame = false;
+
+    public AbstractPartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount,
+            IPartitionWriterFactory pwFactory, RecordDescriptor recordDescriptor) throws HyracksDataException {
+        this.consumerPartitionCount = consumerPartitionCount;
+        pWriters = new IFrameWriter[consumerPartitionCount];
+        appenders = new FrameTupleAppender[consumerPartitionCount];
+        for (int i = 0; i < consumerPartitionCount; ++i) {
+            try {
+                pWriters[i] = pwFactory.createFrameWriter(i);
+                appenders[i] = new FrameTupleAppender();
+            } catch (IOException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+        tupleAccessor = new FrameTupleAccessor(recordDescriptor);
+        this.ctx = ctx;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (int i = 0; i < pWriters.length; ++i) {
+            if (allocatedFrame) {
+                appenders[i].write(pWriters[i], true);
+            }
+            pWriters[i].close();
+        }
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        for (int i = 0; i < pWriters.length; ++i) {
+            pWriters[i].open();
+        }
+    }
+
+    @Override
+    abstract public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+    protected void allocateFrames() throws HyracksDataException {
+        for (int i = 0; i < appenders.length; ++i) {
+            appenders[i].reset(new VSizeFrame(ctx), true);
+        }
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        for (int i = 0; i < appenders.length; ++i) {
+            pWriters[i].fail();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
new file mode 100644
index 0000000..520ddd9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SortMergeFrameReader;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+
+public class MToNRangePartitionMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITupleRangePartitionComputerFactory tprcf;
+    private final RangeId rangeId;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory nkcFactory;
+
+    public MToNRangePartitionMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITupleRangePartitionComputerFactory tprcf, RangeId rangeId, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nkcFactory) {
+        super(spec);
+        this.tprcf = tprcf;
+        this.rangeId = rangeId;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        this.nkcFactory = nkcFactory;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new PartitionRangeDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tprcf, rangeId);
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
+        IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+                sortFields, comparators, nmkComputer, recordDesc, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..6c83abb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SortMergeFrameReader;
+
+public class MToNRangePartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private final ITupleRangePartitionComputerFactory trpcf;
+    private final RangeId rangeId;
+    private final int[] sortFields;
+    private final IBinaryComparatorFactory[] comparatorFactories;
+    private final INormalizedKeyComputerFactory nkcFactory;
+
+    public MToNRangePartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec,
+            ITupleRangePartitionComputerFactory trpcf, RangeId rangeId, int[] sortFields,
+            IBinaryComparatorFactory[] comparatorFactories, INormalizedKeyComputerFactory nkcFactory) {
+        super(spec);
+        this.trpcf = trpcf;
+        this.rangeId = rangeId;
+        this.sortFields = sortFields;
+        this.comparatorFactories = comparatorFactories;
+        this.nkcFactory = nkcFactory;
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        return new PartitionRangeDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, trpcf, rangeId);
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
+        IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+                sortFields, comparators, nmkComputer, recordDesc, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
new file mode 100644
index 0000000..c235afb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.connectors;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.storage.IGrowableIntArray;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+import org.apache.hyracks.dataflow.std.misc.RangeForwardOperatorDescriptor.RangeForwardTaskState;
+import org.apache.hyracks.storage.common.arraylist.IntArrayList;
+
+public class PartitionRangeDataWriter extends AbstractPartitionDataWriter {
+    private final ITupleRangePartitionComputerFactory trpcf;
+    private final RangeId rangeId;
+    private final IGrowableIntArray map;
+    private ITupleRangePartitionComputer tpc;
+
+    public PartitionRangeDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount,
+            IPartitionWriterFactory pwFactory, RecordDescriptor recordDescriptor,
+            ITupleRangePartitionComputerFactory trpcf, RangeId rangeId) throws HyracksDataException {
+        super(ctx, consumerPartitionCount, pwFactory, recordDescriptor);
+        this.trpcf = trpcf;
+        this.rangeId = rangeId;
+        this.map = new IntArrayList(8, 8);
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        super.open();
+        RangeForwardTaskState rangeState = RangeForwardTaskState.getRangeState(rangeId.getId(), ctx);
+        tpc = trpcf.createPartitioner(rangeState.getRangeMap());
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (!allocatedFrame) {
+            allocateFrames();
+            allocatedFrame = true;
+        }
+        tupleAccessor.reset(buffer);
+        int tupleCount = tupleAccessor.getTupleCount();
+        for (int i = 0; i < tupleCount; ++i) {
+            tpc.partition(tupleAccessor, i, consumerPartitionCount, map);
+            for (int h = 0; h < map.size(); ++h) {
+                FrameUtils.appendToWriter(pWriters[map.get(h)], appenders[map.get(h)], tupleAccessor, i);
+            }
+            map.clear();
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/AbstractMergeJoiner.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/AbstractMergeJoiner.java
new file mode 100644
index 0000000..b1b4075
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/AbstractMergeJoiner.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+
+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.std.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.TupleAccessor;
+import org.apache.hyracks.dataflow.std.join.MergeBranchStatus.Stage;
+
+public abstract class AbstractMergeJoiner implements IMergeJoiner {
+
+    public enum TupleStatus {
+        UNKNOWN,
+        LOADED,
+        EMPTY;
+
+        public boolean isLoaded() {
+            return this.equals(LOADED);
+        }
+
+        public boolean isEmpty() {
+            return this.equals(EMPTY);
+        }
+
+        public boolean isKnown() {
+            return !this.equals(UNKNOWN);
+        }
+    }
+
+    protected static final int JOIN_PARTITIONS = 2;
+    protected static final int LEFT_PARTITION = 0;
+    protected static final int RIGHT_PARTITION = 1;
+
+    protected final ByteBuffer[] inputBuffer;
+    protected final FrameTupleAppender resultAppender;
+    protected final ITupleAccessor[] inputAccessor;
+    protected final MergeStatus status;
+
+    private final int partition;
+    private final MergeJoinLocks locks;
+    protected long[] frameCounts = { 0, 0 };
+
+    public AbstractMergeJoiner(IHyracksTaskContext ctx, int partition, MergeStatus status, MergeJoinLocks locks,
+            RecordDescriptor leftRd, RecordDescriptor rightRd) throws HyracksDataException {
+        this.partition = partition;
+        this.status = status;
+        this.locks = locks;
+
+        inputAccessor = new TupleAccessor[JOIN_PARTITIONS];
+        inputAccessor[LEFT_PARTITION] = new TupleAccessor(leftRd);
+        inputAccessor[RIGHT_PARTITION] = new TupleAccessor(rightRd);
+
+        inputBuffer = new ByteBuffer[JOIN_PARTITIONS];
+        inputBuffer[LEFT_PARTITION] = ctx.allocateFrame();
+        inputBuffer[RIGHT_PARTITION] = ctx.allocateFrame();
+
+        // Result
+        resultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+    }
+
+    public void setLeftFrame(ByteBuffer buffer) {
+        setFrame(LEFT_PARTITION, buffer);
+    }
+
+    public void setRightFrame(ByteBuffer buffer) {
+        setFrame(RIGHT_PARTITION, buffer);
+    }
+
+    protected TupleStatus loadMemoryTuple(int branch) {
+        TupleStatus loaded;
+        if (inputAccessor[branch] != null && inputAccessor[branch].exists()) {
+            // Still processing frame.
+            int test = inputAccessor[branch].getTupleCount();
+            loaded = TupleStatus.LOADED;
+        } else if (status.branch[branch].hasMore()) {
+            loaded = TupleStatus.UNKNOWN;
+        } else {
+            // No more frames or tuples to process.
+            loaded = TupleStatus.EMPTY;
+        }
+        return loaded;
+    }
+
+    protected TupleStatus pauseAndLoadRightTuple() {
+        status.continueRightLoad = true;
+        locks.getRight(partition).signal();
+        try {
+            while (status.continueRightLoad
+                    && status.branch[RIGHT_PARTITION].getStatus().isEqualOrBefore(Stage.DATA_PROCESSING)) {
+                locks.getLeft(partition).await();
+            }
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        if (inputAccessor[RIGHT_PARTITION] != null && !inputAccessor[RIGHT_PARTITION].exists()
+                && status.branch[RIGHT_PARTITION].getStatus() == Stage.CLOSED) {
+            status.branch[RIGHT_PARTITION].noMore();
+            return TupleStatus.EMPTY;
+        }
+        return TupleStatus.LOADED;
+    }
+
+    @Override
+    public void setFrame(int branch, ByteBuffer buffer) {
+        inputBuffer[branch].clear();
+        if (inputBuffer[branch].capacity() < buffer.capacity()) {
+            inputBuffer[branch].limit(buffer.capacity());
+        }
+        inputBuffer[branch].put(buffer.array(), 0, buffer.capacity());
+        inputAccessor[branch].reset(inputBuffer[branch]);
+        inputAccessor[branch].next();
+        frameCounts[branch]++;
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
new file mode 100644
index 0000000..49a3763
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public interface IMergeJoinChecker extends Serializable {
+
+    /**
+     * 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(ITupleAccessor accessorLeft, ITupleAccessor accessorRight) 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(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException;
+
+    boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException;
+
+    /**
+     * Check to see if the next right tuple should be loaded during the merge join.
+     * The check is true if the left tuple could match with the next right tuple.
+     * Once the left tuple can no long match, the check returns false.
+     *
+     * @param accessorLeft
+     * @param accessorRight
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkToLoadNextRightTuple(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException;
+
+    boolean checkToSaveInResult(ITupleAccessor accessorLeft, ITupleAccessor accessorRight) throws HyracksDataException;
+
+    boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+            int rightTupleIndex, boolean reversed) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..d7ac550
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMergeJoinCheckerFactory extends Serializable {
+
+    IMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) throws HyracksDataException;
+
+    RangePartitioningType getLeftPartitioningType();
+
+    RangePartitioningType getRightPartitioningType();
+
+    boolean isOrderAsc();
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoiner.java
similarity index 64%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoiner.java
index 500ac03..051d3e0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoiner.java
@@ -16,11 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.dataflow.std.join;
 
-drop dataverse test if exists;
-create dataverse test;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMergeJoiner {
+
+    void processLeftFrame(IFrameWriter writer) throws HyracksDataException;
+
+    void processLeftClose(IFrameWriter writer) throws HyracksDataException;
+
+    void setFrame(int partition, ByteBuffer buffer);
+
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IRunFileStreamStatus.java
similarity index 72%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IRunFileStreamStatus.java
index 500ac03..378ade2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IRunFileStreamStatus.java
@@ -16,11 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.dataflow.std.join;
 
-drop dataverse test if exists;
-create dataverse test;
+public interface IRunFileStreamStatus {
+
+    public boolean isRunFileWriting();
+
+    public void setRunFileWriting(boolean runFileWriting);
+
+    public boolean isRunFileReading();
+
+    public void setRunFileReading(boolean runFileReading);
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeBranchStatus.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeBranchStatus.java
new file mode 100644
index 0000000..4f69189
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeBranchStatus.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.dataflow.std.join.IRunFileStreamStatus;
+
+public class MergeBranchStatus implements IRunFileStreamStatus, Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public enum Stage {
+        UNKNOWN,
+        OPENED,
+        DATA_PROCESSING,
+        JOIN_PROCESSING,
+        CLOSED;
+
+        public boolean isEqualOrBefore(Stage bs) {
+            return this.ordinal() <= bs.ordinal();
+        }
+    }
+
+    private boolean hasMore = true;
+
+    private Stage stage = Stage.UNKNOWN;
+
+    private boolean runFileWriting = false;
+    private boolean runFileReading = false;
+
+    public Stage getStatus() {
+        return stage;
+    }
+
+    public void setStageOpen() {
+        stage = Stage.OPENED;
+    }
+
+    public void setStageData() {
+        stage = Stage.DATA_PROCESSING;
+    }
+
+    public void setStageJoin() {
+        stage = Stage.JOIN_PROCESSING;
+    }
+
+    public void setStageClose() {
+        stage = Stage.CLOSED;
+    }
+
+    public boolean hasMore() {
+        return hasMore;
+    }
+
+    public void noMore() {
+        this.hasMore = false;
+    }
+
+    @Override
+    public boolean isRunFileWriting() {
+        return runFileWriting;
+    }
+
+    @Override
+    public void setRunFileWriting(boolean runFileWriting) {
+        this.runFileWriting = runFileWriting;
+    }
+
+    @Override
+    public boolean isRunFileReading() {
+        return runFileReading;
+    }
+
+    @Override
+    public void setRunFileReading(boolean runFileReading) {
+        this.runFileReading = runFileReading;
+    }
+
+    @Override
+    public String toString() {
+        return "Branch status is " + stage + ": the stream " + (hasMore ? "has more" : "is empty")
+                + " and the run file is " + (runFileWriting ? "WRITING " : "") + (runFileReading ? "READING " : "");
+
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
new file mode 100644
index 0000000..840f396
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class MergeJoinLocks implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private final List<Lock> lock = new ArrayList<>();
+    private final List<Condition> left = new ArrayList<>();
+    private final List<Condition> right = new ArrayList<>();
+
+    public synchronized void setPartitions(int partitions) {
+        for (int i = lock.size(); i < partitions; ++i) {
+            lock.add(new ReentrantLock());
+            left.add(lock.get(i).newCondition());
+            right.add(lock.get(i).newCondition());
+        }
+    }
+
+    public Lock getLock(int partition) {
+        return lock.get(partition);
+    }
+
+    public Condition getLeft(int partition) {
+        return left.get(partition);
+    }
+
+    public Condition getRight(int partition) {
+        return right.get(partition);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
new file mode 100644
index 0000000..c5f612f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+
+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.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.join.MergeBranchStatus.Stage;
+
+/**
+ * The merge join is made up of two operators: left and right.
+ * The right operator loads right stream into memory for the merge process.
+ * The left operator streams the left input and the right memory store to merge and join the data.
+ */
+public class MergeJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int LEFT_ACTIVITY_ID = 0;
+    private static final int RIGHT_ACTIVITY_ID = 1;
+    private final int[] leftKeys;
+    private final int[] rightKeys;
+    private final int memoryForJoin;
+    private final IMergeJoinCheckerFactory mergeJoinCheckerFactory;
+
+    public MergeJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin,
+            RecordDescriptor recordDescriptor, int[] leftKeys, int[] rightKeys,
+            IMergeJoinCheckerFactory mergeJoinCheckerFactory) {
+        super(spec, 2, 1);
+        recordDescriptors[0] = recordDescriptor;
+        this.leftKeys = leftKeys;
+        this.rightKeys = rightKeys;
+        this.memoryForJoin = memoryForJoin;
+        this.mergeJoinCheckerFactory = mergeJoinCheckerFactory;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        MergeJoinLocks locks = new MergeJoinLocks();
+
+        ActivityId leftAid = new ActivityId(odId, LEFT_ACTIVITY_ID);
+        ActivityId rightAid = new ActivityId(odId, RIGHT_ACTIVITY_ID);
+
+        IActivity leftAN = new LeftJoinerActivityNode(leftAid, rightAid, locks);
+        IActivity rightAN = new RightDataActivityNode(rightAid, leftAid, locks);
+
+        builder.addActivity(this, rightAN);
+        builder.addSourceEdge(1, rightAN, 0);
+
+        builder.addActivity(this, leftAN);
+        builder.addSourceEdge(0, leftAN, 0);
+        builder.addTargetEdge(0, leftAN, 0);
+    }
+
+    private class LeftJoinerActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final MergeJoinLocks locks;
+
+        public LeftJoinerActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new LeftJoinerOperator(ctx, partition, inRecordDesc);
+        }
+
+        private class LeftJoinerOperator extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+            private final IHyracksTaskContext ctx;
+            private final int partition;
+            private final RecordDescriptor leftRd;
+            private MergeJoinTaskState state;
+            private boolean first = true;
+            int count = 0;
+
+            public LeftJoinerOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.leftRd = inRecordDesc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    writer.open();
+                    state = new MergeJoinTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));
+                    state.leftRd = leftRd;
+                    ctx.setStateObject(state);
+                    locks.getRight(partition).signal();
+
+                    do {
+                        // Continue after joiner created in right branch.
+                        if (state.joiner == null) {
+                            locks.getLeft(partition).await();
+                        }
+                    } while (state.joiner == null);
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageOpen();
+                    locks.getRight(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+
+                count++;
+                if (first) {
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    state.joiner.setFrame(LEFT_ACTIVITY_ID, buffer);
+                    state.joiner.processLeftFrame(writer);
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[LEFT_ACTIVITY_ID].noMore();
+                    if (state.failed) {
+                        writer.fail();
+                    } else {
+                        state.joiner.processLeftClose(writer);
+                        writer.close();
+                    }
+                    state.status.branch[LEFT_ACTIVITY_ID].setStageClose();
+                    locks.getRight(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+//                System.err.println("Left next calls: " + count);
+            }
+        }
+    }
+
+    private class RightDataActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId joinAid;
+        private final MergeJoinLocks locks;
+
+        public RightDataActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+            super(id);
+            this.joinAid = joinAid;
+            this.locks = locks;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+                throws HyracksDataException {
+            locks.setPartitions(nPartitions);
+            RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            final IMergeJoinChecker mjc = mergeJoinCheckerFactory.createMergeJoinChecker(leftKeys, rightKeys, partition,
+                    ctx);
+            return new RightDataOperator(ctx, partition, inRecordDesc, mjc);
+        }
+
+        private class RightDataOperator extends AbstractUnaryInputSinkOperatorNodePushable {
+
+            private int partition;
+            private IHyracksTaskContext ctx;
+            private final RecordDescriptor rightRd;
+            private final IMergeJoinChecker mjc;
+            private MergeJoinTaskState state;
+            private boolean first = true;
+            int count = 0;
+
+            public RightDataOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc,
+                    IMergeJoinChecker mjc) {
+                this.ctx = ctx;
+                this.partition = partition;
+                this.rightRd = inRecordDesc;
+                this.mjc = mjc;
+            }
+
+            @Override
+            public void open() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    do {
+                        // Wait for the state to be set in the context form Left.
+                        state = (MergeJoinTaskState) ctx.getStateObject(new TaskId(joinAid, partition));
+                        if (state == null) {
+                            locks.getRight(partition).await();
+                        }
+                    } while (state == null);
+                    state.rightRd = rightRd;
+                    state.joiner = new MergeJoiner(ctx, memoryForJoin, partition, state.status, locks, mjc,
+                            state.leftRd, state.rightRd);
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageOpen();
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                locks.getLock(partition).lock();
+                count++;
+                if (first) {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageData();
+                    first = false;
+                }
+                try {
+                    while (!state.status.continueRightLoad
+                            && state.status.branch[LEFT_ACTIVITY_ID].getStatus() != Stage.CLOSED) {
+                        // Wait for the state to request right frame unless left has finished.
+                        locks.getRight(partition).await();
+                    }
+                    state.joiner.setFrame(RIGHT_ACTIVITY_ID, buffer);
+                    state.status.continueRightLoad = false;
+                    locks.getLeft(partition).signal();
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void fail() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.failed = true;
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+            }
+
+            @Override
+            public void close() throws HyracksDataException {
+                locks.getLock(partition).lock();
+                try {
+                    state.status.branch[RIGHT_ACTIVITY_ID].setStageClose();
+                    locks.getLeft(partition).signal();
+                } finally {
+                    locks.getLock(partition).unlock();
+                }
+//                System.err.println("Right next calls: " + count);
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinTaskState.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinTaskState.java
new file mode 100644
index 0000000..4a15a98
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinTaskState.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+
+public class MergeJoinTaskState extends AbstractStateObject {
+    public MergeStatus status;
+    public IMergeJoiner joiner;
+    public boolean failed;
+    public RecordDescriptor leftRd;
+    public RecordDescriptor rightRd;
+
+    public MergeJoinTaskState(JobId jobId, TaskId taskId) {
+        super(jobId, taskId);
+        status = new MergeStatus();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
new file mode 100644
index 0000000..efb3756
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
@@ -0,0 +1,335 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.util.LinkedList;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.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.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.ITuplePointerAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.TupleAccessor;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableDeletableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.structures.RunFilePointer;
+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 MergeJoiner extends AbstractMergeJoiner {
+
+    private static final Logger LOGGER = Logger.getLogger(MergeJoiner.class.getName());
+
+    private final IDeallocatableFramePool framePool;
+    private final IDeletableTupleBufferManager bufferManager;
+    private final ITuplePointerAccessor memoryAccessor;
+    private final LinkedList<TuplePointer> memoryBuffer = new LinkedList<>();
+
+    private int leftStreamIndex;
+    private final RunFileStream runFileStreamOld;
+    private final RunFileStream runFileStream;
+    private ITupleAccessor tmpAccessor;
+    private final RunFilePointer runFilePointer;
+
+    private final IMergeJoinChecker mjc;
+
+    private long joinComparisonCount = 0;
+    private long joinResultCount = 0;
+    private long spillFileCount = 0;
+    private long spillWriteCount = 0;
+    private long spillReadCount = 0;
+    private long spillCount = 0;
+
+    public MergeJoiner(IHyracksTaskContext ctx, int memorySize, int partition, MergeStatus status, MergeJoinLocks locks,
+            IMergeJoinChecker mjc, RecordDescriptor leftRd, RecordDescriptor rightRd) throws HyracksDataException {
+        super(ctx, partition, status, locks, leftRd, rightRd);
+        this.mjc = mjc;
+        tmpAccessor = new TupleAccessor(leftRd);
+        runFilePointer = new RunFilePointer();
+
+        // Memory (right buffer)
+        if (memorySize < 1) {
+            throw new HyracksDataException(
+                    "MergeJoiner does not have enough memory (needs > 0, got " + memorySize + ").");
+        }
+        framePool = new DeallocatableFramePool(ctx, (memorySize) * ctx.getInitialFrameSize());
+        bufferManager = new VariableDeletableTupleMemoryManager(framePool, rightRd);
+        memoryAccessor = bufferManager.createTuplePointerAccessor();
+
+        // Run File and frame cache (left buffer)
+        leftStreamIndex = TupleAccessor.UNSET;
+        runFileStreamOld = new RunFileStream(ctx, "left", status.branch[LEFT_PARTITION]);
+        runFileStream = new RunFileStream(ctx, "left", status.branch[LEFT_PARTITION]);
+
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine(
+                    "MergeJoiner has started partition " + partition + " with " + memorySize + " frames of memory.");
+        }
+    }
+
+    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 removeFromMemory(TuplePointer tp) throws HyracksDataException {
+        memoryBuffer.remove(tp);
+        bufferManager.deleteTuple(tp);
+    }
+
+    private void addToResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+            int rightTupleIndex, IFrameWriter writer) throws HyracksDataException {
+        FrameUtils.appendConcatToWriter(writer, resultAppender, accessorLeft, leftTupleIndex, accessorRight,
+                rightTupleIndex);
+        joinResultCount++;
+    }
+
+    private void flushMemory() throws HyracksDataException {
+        memoryBuffer.clear();
+        bufferManager.reset();
+    }
+
+    // memory management
+    private boolean memoryHasTuples() {
+        return bufferManager.getNumTuples() > 0;
+    }
+
+    /**
+     * Ensures a frame exists for the right branch, either from memory or the run file.
+     *
+     * @throws HyracksDataException
+     */
+    private TupleStatus loadRightTuple() throws HyracksDataException {
+        TupleStatus loaded = loadMemoryTuple(RIGHT_PARTITION);
+        if (loaded == TupleStatus.UNKNOWN) {
+            loaded = pauseAndLoadRightTuple();
+        }
+        return loaded;
+    }
+
+    /**
+     * Ensures a frame exists for the right branch, either from memory or the run file.
+     *
+     * @throws HyracksDataException
+     */
+    private TupleStatus loadLeftTuple() throws HyracksDataException {
+        TupleStatus loaded;
+        if (status.branch[LEFT_PARTITION].isRunFileReading()) {
+            loaded = loadSpilledTuple(LEFT_PARTITION);
+            if (loaded.isEmpty()) {
+                if (status.branch[LEFT_PARTITION].isRunFileWriting() && !status.branch[LEFT_PARTITION].hasMore()) {
+                    unfreezeAndContinue(inputAccessor[LEFT_PARTITION]);
+                } else {
+                    continueStream(inputAccessor[LEFT_PARTITION]);
+                }
+                loaded = loadLeftTuple();
+            }
+        } else {
+            loaded = loadMemoryTuple(LEFT_PARTITION);
+        }
+        return loaded;
+    }
+
+    private TupleStatus loadSpilledTuple(int partition) throws HyracksDataException {
+        if (!inputAccessor[partition].exists()) {
+            runFileStream.loadNextBuffer(tmpAccessor);
+            if (!runFileStreamOld.loadNextBuffer(inputAccessor[partition])) {
+                return TupleStatus.EMPTY;
+            }
+        }
+        return TupleStatus.LOADED;
+    }
+
+    /**
+     * Left
+     *
+     * @throws HyracksDataException
+     */
+    @Override
+    public void processLeftFrame(IFrameWriter writer) throws HyracksDataException {
+        TupleStatus leftTs = loadLeftTuple();
+        TupleStatus rightTs = loadRightTuple();
+        while (leftTs.isLoaded() && (status.branch[RIGHT_PARTITION].hasMore() || memoryHasTuples())) {
+            if (status.branch[LEFT_PARTITION].isRunFileWriting()) {
+                // Left side from disk
+                leftTs = processLeftTupleSpill(writer);
+            } else if (rightTs.isLoaded()
+                    && mjc.checkToLoadNextRightTuple(inputAccessor[LEFT_PARTITION], inputAccessor[RIGHT_PARTITION])) {
+                // Right side from stream
+                processRightTuple();
+                rightTs = loadRightTuple();
+            } else {
+                // Left side from stream
+                processLeftTuple(writer);
+                leftTs = loadLeftTuple();
+            }
+        }
+    }
+
+    @Override
+    public void processLeftClose(IFrameWriter writer) throws HyracksDataException {
+        if (status.branch[LEFT_PARTITION].isRunFileWriting()) {
+            unfreezeAndContinue(inputAccessor[LEFT_PARTITION]);
+        }
+        processLeftFrame(writer);
+        resultAppender.write(writer, true);
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("MergeJoiner statitics: " + joinComparisonCount + " comparisons, " + joinResultCount
+                    + " results, " + spillCount + " spills, " + runFileStreamOld.getFileCount() + " files, "
+                    + runFileStreamOld.getWriteCount() + " spill frames written, " + runFileStreamOld.getReadCount()
+                    + " spill frames read.");
+        }
+    }
+
+    private TupleStatus processLeftTupleSpill(IFrameWriter writer) throws HyracksDataException {
+        //        System.err.print("Spill ");
+
+        runFileStreamOld.addToRunFile(inputAccessor[LEFT_PARTITION]);
+        if (true) {
+            runFileStream.addToRunFile(inputAccessor[LEFT_PARTITION]);
+        }
+
+        processLeftTuple(writer);
+
+        // Memory is empty and we can start processing the run file.
+        if (!memoryHasTuples() && status.branch[LEFT_PARTITION].isRunFileWriting()) {
+            unfreezeAndContinue(inputAccessor[LEFT_PARTITION]);
+        }
+        return loadLeftTuple();
+    }
+
+    private void processLeftTuple(IFrameWriter writer) throws HyracksDataException {
+        //        TuplePrinterUtil.printTuple("Left", inputAccessor[LEFT]);
+        // Check against memory (right)
+        if (memoryHasTuples()) {
+            for (int i = memoryBuffer.size() - 1; i > -1; --i) {
+                memoryAccessor.reset(memoryBuffer.get(i));
+                if (mjc.checkToSaveInResult(inputAccessor[LEFT_PARTITION], inputAccessor[LEFT_PARTITION].getTupleId(),
+                        memoryAccessor, memoryBuffer.get(i).getTupleIndex(), false)) {
+                    // add to result
+                    addToResult(inputAccessor[LEFT_PARTITION], inputAccessor[LEFT_PARTITION].getTupleId(),
+                            memoryAccessor, memoryBuffer.get(i).getTupleIndex(), writer);
+                }
+                joinComparisonCount++;
+                if (mjc.checkToRemoveInMemory(inputAccessor[LEFT_PARTITION], inputAccessor[LEFT_PARTITION].getTupleId(),
+                        memoryAccessor, memoryBuffer.get(i).getTupleIndex())) {
+                    // remove from memory
+                    //                    TuplePrinterUtil.printTuple("Remove Memory", memoryAccessor, memoryBuffer.get(i).getTupleIndex());
+                    removeFromMemory(memoryBuffer.get(i));
+                }
+            }
+        }
+        inputAccessor[LEFT_PARTITION].next();
+    }
+
+    private void processRightTuple() throws HyracksDataException {
+        // append to memory
+        if (mjc.checkToSaveInMemory(inputAccessor[LEFT_PARTITION], inputAccessor[RIGHT_PARTITION])) {
+            if (!addToMemory(inputAccessor[RIGHT_PARTITION])) {
+                // go to log saving state
+                freezeAndSpill();
+                return;
+            }
+        }
+        //        TuplePrinterUtil.printTuple("Memory", inputAccessor[RIGHT]);
+        inputAccessor[RIGHT_PARTITION].next();
+    }
+
+    private void freezeAndSpill() throws HyracksDataException {
+        //        System.err.println("freezeAndSpill");
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("freeze snapshot: " + frameCounts[RIGHT_PARTITION] + " right, " + frameCounts[LEFT_PARTITION]
+                    + " left, " + joinComparisonCount + " comparisons, " + joinResultCount + " results, ["
+                    + bufferManager.getNumTuples() + " tuples memory].");
+        }
+
+        if (runFilePointer.getFileOffset() > 0) {
+
+        } else {
+            runFilePointer.reset(0, 0);
+            runFileStream.startRunFileWriting();
+        }
+        runFileStreamOld.startRunFileWriting();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine(
+                    "Memory is full. Freezing the right branch. (memory tuples: " + bufferManager.getNumTuples() + ")");
+        }
+        spillCount++;
+    }
+
+    private void continueStream(ITupleAccessor accessor) throws HyracksDataException {
+        //        System.err.println("continueStream");
+
+        runFileStreamOld.closeRunFileReading();
+        accessor.reset(inputBuffer[LEFT_PARTITION]);
+        accessor.setTupleId(leftStreamIndex);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Continue with left stream.");
+        }
+    }
+
+    private void unfreezeAndContinue(ITupleAccessor accessor) throws HyracksDataException {
+        //        System.err.println("unfreezeAndContinue");
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("snapshot: " + frameCounts[RIGHT_PARTITION] + " right, " + frameCounts[LEFT_PARTITION]
+                    + " left, " + joinComparisonCount + " comparisons, " + joinResultCount + " results, ["
+                    + bufferManager.getNumTuples() + " tuples memory, " + spillCount + " spills, "
+                    + (runFileStreamOld.getFileCount() - spillFileCount) + " files, "
+                    + (runFileStreamOld.getWriteCount() - spillWriteCount) + " written, "
+                    + (runFileStreamOld.getReadCount() - spillReadCount) + " read].");
+            spillFileCount = runFileStreamOld.getFileCount();
+            spillReadCount = runFileStreamOld.getReadCount();
+            spillWriteCount = runFileStreamOld.getWriteCount();
+        }
+
+        runFileStreamOld.flushAndStopRunFile(accessor);
+        runFileStream.flushAndStopRunFile(accessor);
+        flushMemory();
+        if (!status.branch[LEFT_PARTITION].isRunFileReading()) {
+            leftStreamIndex = accessor.getTupleId();
+        }
+        runFileStreamOld.startReadingRunFile(accessor);
+
+        runFileStream.resetReadPointer(runFilePointer.getFileOffset());
+        accessor.setTupleId(runFilePointer.getTupleIndex());
+        runFileStream.startReadingRunFile(accessor);
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("Unfreezing right partition.");
+        }
+
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
similarity index 62%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
copy to hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
index 500ac03..446ebc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/interval_functions/interval_functions.1.ddl.aql
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
@@ -16,11 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/*
- * Description      :   Check temporal functions for interval
- * Expected Result  :   Success
- * Date             :   2nd Nov, 2012
- */
+package org.apache.hyracks.dataflow.std.join;
 
-drop dataverse test if exists;
-create dataverse test;
+import java.io.Serializable;
+
+public class MergeStatus implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public boolean reloadingLeftFrame = false;
+    public boolean continueRightLoad = false;
+
+    public MergeBranchStatus[] branch = new MergeBranchStatus[2];
+
+    public MergeStatus() {
+        branch[0] = new MergeBranchStatus();
+        branch[1] = new MergeBranchStatus();
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
new file mode 100644
index 0000000..9a50d0c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
+
+public class NaturalMergeJoinChecker implements IMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+    private final FrameTuplePairComparator comparator;
+
+    public NaturalMergeJoinChecker(FrameTuplePairComparator comparator) {
+        this.comparator = comparator;
+    }
+
+    @Override
+    public boolean checkToSaveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        int c = comparator.compare(accessorLeft, accessorLeft.getTupleId(), accessorRight, accessorRight.getTupleId());
+        return c == 0;
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        return checkToRemoveInMemory(accessorLeft, accessorLeft.getTupleId(), accessorRight, accessorRight.getTupleId());
+    }
+
+    @Override
+    public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        int c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+        return c < 0;
+    }
+
+    @Override
+    public boolean checkToLoadNextRightTuple(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        int c = comparator.compare(accessorLeft, accessorLeft.getTupleId(), accessorRight, accessorRight.getTupleId());
+        return c <= 0;
+    }
+
+    @Override
+    public boolean checkToSaveInResult(ITupleAccessor accessorLeft, ITupleAccessor accessorRight)
+            throws HyracksDataException {
+        int c = comparator.compare(accessorLeft, accessorLeft.getTupleId(), accessorRight, accessorRight.getTupleId());
+        return c == 0;
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex, boolean reversed) throws HyracksDataException {
+        int c;
+        if (reversed) {
+            c = comparator.compare(accessorRight, rightTupleIndex, accessorLeft, leftTupleIndex);
+        } else {
+            c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+        }
+        return c == 0;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..abdadb6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.dataflow.std.util.FrameTuplePairComparator;
+
+public class NaturalMergeJoinCheckerFactory implements IMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+    private IBinaryComparatorFactory[] comparatorFactories;
+
+    public NaturalMergeJoinCheckerFactory(IBinaryComparatorFactory[] comparatorFactories) {
+        this.comparatorFactories = comparatorFactories;
+    }
+
+    @Override
+    public IMergeJoinChecker createMergeJoinChecker(int[] keys0, int[] keys1, int partition, IHyracksTaskContext ctx) {
+        final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+        for (int i = 0; i < comparatorFactories.length; ++i) {
+            comparators[i] = comparatorFactories[i].createBinaryComparator();
+        }
+        FrameTuplePairComparator ftp = new FrameTuplePairComparator(keys0, keys1, comparators);
+        return new NaturalMergeJoinChecker(ftp);
+    }
+
+    @Override
+    public RangePartitioningType getLeftPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public RangePartitioningType getRightPartitioningType() {
+        return RangePartitioningType.PROJECT;
+    }
+
+    @Override
+    public boolean isOrderAsc() {
+        return false;
+    }
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
index 202aac6..5cc36ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -20,6 +20,8 @@
 
 import java.io.DataOutput;
 import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import org.apache.hyracks.api.comm.IFrame;
 import org.apache.hyracks.api.comm.IFrameWriter;
@@ -43,6 +45,8 @@
 import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
 
 public class NestedLoopJoin {
+    private static final Logger LOGGER = Logger.getLogger(NestedLoopJoin.class.getName());
+
     private final FrameTupleAccessor accessorInner;
     private final FrameTupleAccessor accessorOuter;
     private final FrameTupleAppender appender;
@@ -58,6 +62,12 @@
     private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
     private BufferInfo tempInfo = new BufferInfo(null, -1, -1);
 
+    private final int partition;
+    private long joinComparisonCount = 0;
+    private long joinResultCount = 0;
+    private long spillWriteCount = 0;
+    private long spillReadCount = 0;
+
     public NestedLoopJoin(IHyracksTaskContext ctx, FrameTupleAccessor accessorOuter, FrameTupleAccessor accessorInner,
             ITuplePairComparator comparatorsOuter2Inner, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
             IMissingWriter[] missingWriters) throws HyracksDataException {
@@ -95,10 +105,13 @@
                 .createManagedWorkspaceFile(this.getClass().getSimpleName() + this.toString());
         runFileWriter = new RunFileWriter(file, ctx.getIOManager());
         runFileWriter.open();
+
+        partition = ctx.getTaskAttemptId().getTaskId().getPartition();
     }
 
     public void cache(ByteBuffer buffer) throws HyracksDataException {
         runFileWriter.nextFrame(buffer);
+        spillWriteCount++;
     }
 
     public void join(ByteBuffer outerBuffer, IFrameWriter writer) throws HyracksDataException {
@@ -109,6 +122,7 @@
                 for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
                     blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
                 }
+                spillReadCount++;
             }
             runFileReader.close();
             outerBufferMngr.reset();
@@ -135,6 +149,7 @@
                     matchFound = true;
                     appendToResults(i, j, writer);
                 }
+                joinComparisonCount++;
             }
 
             if (!matchFound && isLeftOuter) {
@@ -149,9 +164,9 @@
 
     private boolean evaluatePredicate(int tIx1, int tIx2) {
         if (isReversed) { //Role Reversal Optimization is triggered
-            return ((predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1));
+            return (predEvaluator == null) || predEvaluator.evaluate(accessorInner, tIx2, accessorOuter, tIx1);
         } else {
-            return ((predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2));
+            return (predEvaluator == null) || predEvaluator.evaluate(accessorOuter, tIx1, accessorInner, tIx2);
         }
     }
 
@@ -166,6 +181,7 @@
     private void appendResultToFrame(FrameTupleAccessor accessor1, int tupleId1, FrameTupleAccessor accessor2,
             int tupleId2, IFrameWriter writer) throws HyracksDataException {
         FrameUtils.appendConcatToWriter(writer, appender, accessor1, tupleId1, accessor2, tupleId2);
+        joinResultCount++;
     }
 
     public void closeCache() throws HyracksDataException {
@@ -181,11 +197,18 @@
             for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
                 blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
             }
+            spillReadCount++;
         }
         runFileReader.close();
         outerBufferMngr.reset();
 
         appender.write(writer, true);
+
+        if (LOGGER.isLoggable(Level.WARNING)) {
+            LOGGER.warning("NestedLoopJoin statitics: " + partition + " partition, " + joinComparisonCount
+                    + " comparisons, " + joinResultCount + " results, " + spillWriteCount + " frames written, "
+                    + spillReadCount + " frames read.");
+        }
     }
 
     private int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/RunFileStream.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/RunFileStream.java
new file mode 100644
index 0000000..f50c34a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/RunFileStream.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.join;
+
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import org.apache.hyracks.api.comm.IFrame;
+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;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.RunFilePointer;
+
+public class RunFileStream {
+
+    private static final Logger LOGGER = Logger.getLogger(RunFileStream.class.getName());
+
+    private final String key;
+    private final IFrame runFileBuffer;
+    private final IFrameTupleAppender runFileAppender;
+    private RunFileWriter runFileWriter;
+    private RunFileReader runFileReader;
+    private final IRunFileStreamStatus status;
+
+    private final IHyracksTaskContext ctx;
+
+    private long runFileCounter = 0;
+    private long readCount = 0;
+    private long writeCount = 0;
+    private long tupleCount = 0;
+
+    /**
+     * The RunFileSream uses two frames to buffer read and write operations.
+     *
+     * @param ctx
+     * @param key
+     * @param status
+     * @throws HyracksDataException
+     */
+    public RunFileStream(IHyracksTaskContext ctx, String key, IRunFileStreamStatus status) throws HyracksDataException {
+        this.ctx = ctx;
+        this.key = key;
+        this.status = status;
+
+        runFileBuffer = new VSizeFrame(ctx);
+        runFileAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+    }
+
+    public long getFileCount() {
+        return runFileCounter;
+    }
+
+    public long getReadCount() {
+        return readCount;
+    }
+
+    public long getWriteCount() {
+        return writeCount;
+    }
+
+    public void startRunFileWriting() throws HyracksDataException {
+        runFileCounter++;
+
+        status.setRunFileWriting(true);
+        String prefix = this.getClass().getSimpleName() + '-' + key + '-' + Long.toString(runFileCounter) + '-'
+                + this.toString();
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(prefix);
+        runFileWriter = new RunFileWriter(file, ctx.getIOManager());
+        runFileWriter.open();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("A new run file has been started (key: " + key + ", number: " + runFileCounter + ", file: "
+                    + file + ").");
+        }
+    }
+
+    public void resumeRunFileWriting() throws HyracksDataException {
+        status.setRunFileWriting(true);
+        String prefix = this.getClass().getSimpleName() + '-' + key + '-' + Long.toString(runFileCounter) + '-'
+                + this.toString();
+        FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(prefix);
+        runFileWriter = new RunFileWriter(file, ctx.getIOManager());
+        runFileWriter.open();
+        if (LOGGER.isLoggable(Level.FINE)) {
+            LOGGER.fine("A new run file has been started (key: " + key + ", number: " + runFileCounter + ", file: "
+                    + file + ").");
+        }
+    }
+
+    public void addToRunFile(ITupleAccessor accessor) throws HyracksDataException {
+        int idx = accessor.getTupleId();
+        if (!runFileAppender.append(accessor, idx)) {
+            runFileAppender.write(runFileWriter, true);
+            writeCount++;
+            runFileAppender.append(accessor, idx);
+            tupleCount = 0;
+        }
+        tupleCount++;
+    }
+
+    public void startReadingRunFile(ITupleAccessor accessor) throws HyracksDataException {
+        status.setRunFileReading(true);
+
+        // Create reader
+        runFileReader = runFileWriter.createReader();
+        runFileReader.open();
+
+        // Load first frame
+        loadNextBuffer(accessor);
+    }
+
+//    public void resetReader(ITupleAccessor accessor) throws HyracksDataException {
+//        if (status.isRunFileWriting()) {
+//            flushAndStopRunFile(accessor);
+//            startReadingRunFile(accessor);
+//        } else {
+//            runFileReader.reset();
+//
+//            // Load first frame
+//            loadNextBuffer(accessor);
+//        }
+//    }
+
+    public boolean loadNextBuffer(ITupleAccessor accessor) throws HyracksDataException {
+        if (runFileReader.nextFrame(runFileBuffer)) {
+            accessor.reset(runFileBuffer.getBuffer());
+            accessor.next();
+            readCount++;
+            return true;
+        }
+        return false;
+    }
+
+    public long getReadPointer() throws HyracksDataException {
+        return runFileReader.getReadPointer();
+    }
+
+    public void resetReadPointer(long fileOffset) throws HyracksDataException {
+        runFileReader.reset(fileOffset);
+    }
+
+    public void flushAndStopRunFile(ITupleAccessor accessor) throws HyracksDataException {
+        status.setRunFileWriting(false);
+
+        // Copy left over tuples into new run file.
+        if (status.isRunFileReading()) {
+            if (!accessor.exists()) {
+                loadNextBuffer(accessor);
+            }
+            while (accessor.exists()) {
+                addToRunFile(accessor);
+                accessor.next();
+                if (!accessor.exists()) {
+                    loadNextBuffer(accessor);
+                }
+            }
+            runFileReader.close();
+        }
+
+        // Flush buffer.
+        if (runFileAppender.getTupleCount() > 0) {
+            runFileAppender.write(runFileWriter, true);
+            writeCount++;
+        }
+    }
+
+    public void closeRunFileReading() throws HyracksDataException {
+        status.setRunFileReading(false);
+        runFileReader.close();
+    }
+
+    public void close() throws HyracksDataException {
+        if (runFileReader != null) {
+            runFileReader.close();
+        }
+        if (runFileWriter != null) {
+            runFileWriter.close();
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/TuplePrinterUtil.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/TuplePrinterUtil.java
new file mode 100644
index 0000000..1e29ede
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/TuplePrinterUtil.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.hyracks.dataflow.std.join;
+
+import java.util.Arrays;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.buffermanager.ITupleAccessor;
+
+public class TuplePrinterUtil {
+
+    private TuplePrinterUtil() {
+    }
+
+    public static void printTuple(String message, ITupleAccessor accessor) throws HyracksDataException {
+        if (accessor.exists()) {
+            printTuple(message, accessor, accessor.getTupleId());
+        } else {
+            System.err.print(String.format("%1$-" + 15 + "s", message) + " --");
+            System.err.print("no tuple");
+            System.err.println();
+        }
+    }
+
+    public static void printTuple(String message, IFrameTupleAccessor accessor, int tupleId)
+            throws HyracksDataException {
+        System.err.print(String.format("%1$-" + 15 + "s", message) + " --");
+        int fields = accessor.getFieldCount();
+        for (int i = 0; i < fields; ++i) {
+            System.err.print(" " + i + ": ");
+            int fieldStartOffset = accessor.getFieldStartOffset(tupleId, i);
+            int fieldSlotsLength = accessor.getFieldSlotsLength();
+            int tupleStartOffset = accessor.getTupleStartOffset(tupleId);
+
+            int start = fieldStartOffset + fieldSlotsLength + tupleStartOffset;
+            int end = start + accessor.getFieldLength(tupleId, i);
+            System.err.print(Arrays.toString(Arrays.copyOfRange(accessor.getBuffer().array(), start, end)));
+        }
+        System.err.println();
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/RangeForwardOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/RangeForwardOperatorDescriptor.java
new file mode 100644
index 0000000..0e4bc4d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/RangeForwardOperatorDescriptor.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.std.misc;
+
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+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.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.RangeId;
+
+public class RangeForwardOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int RANGE_FORWARD_ACTIVITY_ID = 0;
+    private static final int RANGE_WRITER_ACTIVITY_ID = 1;
+
+    private final RangeId rangeId;
+    private final IRangeMap rangeMap;
+
+    public RangeForwardOperatorDescriptor(IOperatorDescriptorRegistry spec, RangeId rangeId, IRangeMap rangeMap,
+            RecordDescriptor recordDescriptor) {
+        super(spec, 1, 1);
+        this.rangeId = rangeId;
+        this.rangeMap = rangeMap;
+        recordDescriptors[0] = recordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ForwardActivityNode fan = new ForwardActivityNode(new ActivityId(odId, RANGE_FORWARD_ACTIVITY_ID));
+        builder.addActivity(this, fan);
+        builder.addSourceEdge(0, fan, 0);
+        builder.addTargetEdge(0, fan, 0);
+    }
+
+    public static class RangeForwardTaskState extends AbstractStateObject {
+        private IRangeMap rangeMap;
+
+        public RangeForwardTaskState(JobId jobId, RangeId rangeId, IRangeMap rangeMap) {
+            super(jobId, rangeId);
+            this.rangeMap = rangeMap;
+        }
+
+        public IRangeMap getRangeMap() {
+            return rangeMap;
+        }
+
+        public static RangeForwardTaskState getRangeState(int rangeId, IHyracksTaskContext ctx)
+                throws HyracksDataException {
+            RangeId rangeIdObject = new RangeId(rangeId, ctx);
+            RangeForwardTaskState rangeState = (RangeForwardTaskState) ctx.getStateObject(rangeIdObject);
+            if (rangeState == null) {
+                throw new HyracksDataException("Range state is missing for " + rangeIdObject + ".");
+            }
+            return rangeState;
+        }
+    }
+
+    private final class ForwardActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public ForwardActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private RangeForwardTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = new RangeForwardTaskState(ctx.getJobletContext().getJobId(),
+                            new RangeId(rangeId.getId(), ctx), rangeMap);
+                    ctx.setStateObject(state);
+                    writer.open();
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer bufferAccessor) throws HyracksDataException {
+                    FrameUtils.flushFrame(bufferAccessor, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    writer.close();
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+        }
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
index 77d5d49..d099e99 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
@@ -157,6 +157,17 @@
         return getFrameCount() > 0;
     }
 
+    public int getTuple(int ptr, FrameTupleAccessor tupleAccessor) {
+        int i = tPointers[ptr * PTR_SIZE + ID_FRAMEID];
+        bufferManager.getFrame(i, info);
+        tupleAccessor.reset(info.getBuffer(), info.getStartOffset(), info.getLength());
+        return tPointers[ptr * PTR_SIZE + ID_TUPLE_START];
+    }
+
+    public int getTupleCount() {
+        return tupleCount;
+    }
+
     @Override
     public int flush(IFrameWriter writer) throws HyracksDataException {
         outputAppender.reset(outputFrame, true);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
index 7d4db64..c143eaf 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
@@ -19,9 +19,14 @@
 
 package org.apache.hyracks.dataflow.std.sort.util;
 
+import java.io.PrintStream;
 import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.PriorityQueue;
 
-import org.apache.hyracks.api.comm.FrameHelper;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.MutablePair;
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -31,59 +36,155 @@
  * This is a special frame which is used in TupleMemoryBuffer.
  * This frame has a special structure to organize the deleted spaces.
  * Specifically, the endOffset of the deleted tuple will be set as negative number.
- * And we add a special <code>deleted_space</code> field at the last 4 bytes to remember how many bytes has been deleted.
+ * And we add a special <code>deleted_space</code> field at the last 4 bytes to
+ * remember how many bytes has been deleted.
+ * The offsets also store both the start and end values because tuples may be out of
+ * after several add, remove and reorganize operations.
+ * A frame is formatted with tuple data concatenated starting at offset 0,
+ * one tuple after another.
+ * Offset FS - 4 holds an int indicating the amount of <code>deleted_space</code> in the frame.
+ * Offset FS - 4 holds an int indicating the number of tuples (N) in the frame.
+ * FS - ((i + 1) * (4 + 4)) for i from 0 to N - 1 holds an two ints indicating
+ * the offset and length of the (i + 1)^th tuple.
+ * The tuple references are organized as a sequence of ints indicating the start of the field
+ * followed by the length of each tuple.
+ * The offset if the start of the frame.
+ * The tuple has been deleted if the length is 0.
+ *
+ * <pre>
+ * [ *tuple_1_bytes*,
+ *   *tuple_2_bytes*,
+ *   ...
+ *   int length, int offset, # tuple 2
+ *   int length, int offset, # tuple 1
+ *   int tuple_append,
+ *   int next_index,
+ *   int deleted_space,
+ *   int index_count,
+ * ]
+ * </pre>
+ *
+ * <pre>
+ * [ *tuple_1_bytes*,
+ *   *tuple_2_bytes*,
+ *   ...
+ *   int end_offset,
+ *   int offset, # tuple 2
+ *   int offset, # tuple 1
+ *   int deleted_space,
+ *   int tuple_count,
+ * ]
+ * </pre>
  */
 public class DeletableFrameTupleAppender implements IAppendDeletableFrameTupleAccessor {
 
+    public static final Comparator<Pair<Integer, Integer>> INDEX_OFFSET_ASC_COMPARATOR = new Comparator<Pair<Integer, Integer>>() {
+        @Override
+        public int compare(Pair<Integer, Integer> p1, Pair<Integer, Integer> p2) {
+            return p1.getValue() - p2.getValue();
+        }
+
+    };
+
+    private static final int SIZE_INDEX_COUNT = 4;
     private static final int SIZE_DELETED_SPACE = 4;
+    private static final int SIZE_NEXT_INDEX = 4;
+    private static final int SIZE_TUPLE_APPEND = 4;
+
+    private static final int SIZE_START_OFFSET = 4;
+    private static final int SIZE_END_OFFSET = 4;
+    private static final int SIZE_OFFSET_GROUP = SIZE_END_OFFSET + SIZE_START_OFFSET;
+
     private final RecordDescriptor recordDescriptor;
     private ByteBuffer buffer;
-    private int tupleCountOffset;
-    private int tupleCount;
-    private int freeDataEndOffset;
+    private int indexSlotsOffset;
+    private int indexCount;
+    private int tupleAppend;
     private int deletedSpace;
-    private byte[] array;   // to speed up the array visit a little
+    private int nextIndex;
+    private byte[] array; // to speed up the array visit a little
+    private IntegerPairPool ipp = new IntegerPairPool();
+
+    private final PriorityQueue<IntegerPair> reorganizeQueue;
 
     public DeletableFrameTupleAppender(RecordDescriptor recordDescriptor) {
         this.recordDescriptor = recordDescriptor;
+        reorganizeQueue = new PriorityQueue<>(16, IntegerPair.RIGHT_ASC_COMPARATOR);
     }
 
-    private int getTupleCountOffset() {
-        return FrameHelper.getTupleCountOffset(buffer.capacity()) - SIZE_DELETED_SPACE;
+    private int getIndexCount() {
+        return IntSerDeUtils.getInt(array, getIndexCountOffset());
     }
 
-    private int getFreeDataEndOffset() {
-        return tupleCount == 0 ? 0 : Math.abs(IntSerDeUtils.getInt(array, tupleCountOffset - tupleCount * 4));
+    private void setIndexCount(int count) {
+        IntSerDeUtils.putInt(array, getIndexCountOffset(), count);
     }
 
-    private void setFreeDataEndOffset(int offset) {
-        assert (offset >= 0);
-        IntSerDeUtils.putInt(array, tupleCountOffset - tupleCount * 4, offset);
-    }
-
-    private void setTupleCount(int count) {
-        IntSerDeUtils.putInt(array, tupleCountOffset, count);
-    }
-
-    private void setDeleteSpace(int count) {
-        IntSerDeUtils.putInt(array, buffer.capacity() - SIZE_DELETED_SPACE, count);
-    }
-
-    private int getPhysicalTupleCount() {
-        return IntSerDeUtils.getInt(array, tupleCountOffset);
+    private int getIndexCountOffset() {
+        return buffer.capacity() - SIZE_INDEX_COUNT;
     }
 
     private int getDeletedSpace() {
-        return IntSerDeUtils.getInt(array, buffer.capacity() - SIZE_DELETED_SPACE);
+        return IntSerDeUtils.getInt(array, getDeletedSpaceOffset());
+    }
+
+    private void setDeletedSpace(int space) {
+        IntSerDeUtils.putInt(array, getDeletedSpaceOffset(), space);
+    }
+
+    private int getDeletedSpaceOffset() {
+        return getIndexCountOffset() - SIZE_DELETED_SPACE;
+    }
+
+    private int getNextIndex() {
+        return IntSerDeUtils.getInt(array, getNextIndexOffset());
+    }
+
+    private void setNextIndex(int index) {
+        IntSerDeUtils.putInt(array, getNextIndexOffset(), index);
+    }
+
+    private int getNextIndexOffset() {
+        return getDeletedSpaceOffset() - SIZE_NEXT_INDEX;
+    }
+
+    private int getAndUpdateNextIndex() {
+        int index = nextIndex;
+        nextIndex = index + 1;
+        while (nextIndex < indexCount) {
+            if (getTupleEndOffset(nextIndex) <= 0) {
+                break;
+            }
+            nextIndex++;
+        }
+        setNextIndex(nextIndex);
+        return index;
+    }
+
+    private int getTupleAppend() {
+        return IntSerDeUtils.getInt(array, getTupleAppendOffset());
+    }
+
+    private void setTupleAppend(int offset) {
+        IntSerDeUtils.putInt(array, getTupleAppendOffset(), offset);
+    }
+
+    private int getTupleAppendOffset() {
+        return getNextIndexOffset() - SIZE_TUPLE_APPEND;
+    }
+
+    private int getIndexSlotOffset() {
+        return getTupleAppendOffset();
     }
 
     @Override
     public void clear(ByteBuffer buffer) throws HyracksDataException {
         this.buffer = buffer;
         this.array = buffer.array();
-        tupleCountOffset = getTupleCountOffset();
-        setTupleCount(0);
-        setDeleteSpace(0);
+        setIndexCount(0);
+        setDeletedSpace(0);
+        setNextIndex(0);
+        setTupleAppend(0);
         resetCounts();
     }
 
@@ -91,14 +192,15 @@
     public void reset(ByteBuffer buffer) {
         this.buffer = buffer;
         this.array = buffer.array();
-        tupleCountOffset = getTupleCountOffset();
         resetCounts();
     }
 
     private void resetCounts() {
+        indexSlotsOffset = getIndexSlotOffset();
         deletedSpace = getDeletedSpace();
-        tupleCount = getPhysicalTupleCount();
-        freeDataEndOffset = getFreeDataEndOffset();
+        indexCount = getIndexCount();
+        tupleAppend = getTupleAppend();
+        nextIndex = getNextIndex();
     }
 
     /**
@@ -115,11 +217,18 @@
         byte[] src = tupleAccessor.getBuffer().array();
         int tStartOffset = tupleAccessor.getTupleStartOffset(tIndex);
         int length = tupleAccessor.getTupleLength(tIndex);
-        System.arraycopy(src, tStartOffset, array, freeDataEndOffset, length);
-        setTupleCount(++tupleCount);
-        freeDataEndOffset += length;
-        setFreeDataEndOffset(freeDataEndOffset);
-        return tupleCount - 1;
+        System.arraycopy(src, tStartOffset, array, tupleAppend, length);
+        int index = getAndUpdateNextIndex();
+        if (index < indexCount) {
+            // Don't change index count
+        } else {
+            // Increment count
+            setIndexCount(++indexCount);
+        }
+        setTupleOffsets(index, tupleAppend, length);
+        tupleAppend += length;
+        setTupleAppend(tupleAppend);
+        return index;
     }
 
     @Override
@@ -128,7 +237,11 @@
         if (endOffset > 0) {
             setTupleEndOffset(tupleIndex, -endOffset);
             deletedSpace += endOffset - getTupleStartOffset(tupleIndex);
-            setDeleteSpace(deletedSpace);
+            setDeletedSpace(deletedSpace);
+            if (nextIndex > tupleIndex) {
+                nextIndex = tupleIndex;
+                setNextIndex(nextIndex);
+            }
         }
     }
 
@@ -139,36 +252,58 @@
         }
         reclaimDeletedEnding();
 
-        freeDataEndOffset = 0;
-        int endOffset = 0;
-        for (int i = 0; i < tupleCount; i++) {
-            int startOffset = Math.abs(endOffset);
+        // Build reorganize queue
+        IntegerPair ip;
+        int endOffset;
+        int startOffset;
+        for (int i = 0; i < indexCount; i++) {
             endOffset = getTupleEndOffset(i);
+            if (endOffset > 0) {
+                ip = ipp.takeOne();
+                ip.reset(i, getTupleStartOffset(i));
+                reorganizeQueue.add(ip);
+            }
+        }
+
+        int index;
+        tupleAppend = 0;
+        while (!reorganizeQueue.isEmpty()) {
+            ip = reorganizeQueue.remove();
+            index = ip.getLeft();
+            startOffset = getTupleStartOffset(index);
+            endOffset = getTupleEndOffset(index);
             if (endOffset >= 0) {
                 int length = endOffset - startOffset;
-                assert ( length >= 0);
-                if (freeDataEndOffset != startOffset) {
-                    System.arraycopy(array, startOffset, array, freeDataEndOffset, length);
+                assert length >= 0;
+                if (tupleAppend != startOffset) {
+                    System.arraycopy(array, startOffset, array, tupleAppend, length);
                 }
-                freeDataEndOffset += length;
+                setTupleOffsets(index, tupleAppend, length);
+                tupleAppend += length;
             }
-            setTupleEndOffset(i, freeDataEndOffset);
+            ipp.giveBack(ip);
         }
-        setFreeDataEndOffset(freeDataEndOffset);
+        setTupleAppend(tupleAppend);
         deletedSpace = 0;
-        setDeleteSpace(0);
+        setDeletedSpace(0);
+
+        // Clean up
+        reorganizeQueue.clear();
     }
 
     private void reclaimDeletedEnding() {
-        for (int i = tupleCount - 1; i >= 0; i--) {
+        for (int i = indexCount - 1; i >= 0; i--) {
             int endOffset = getTupleEndOffset(i);
-            if (endOffset < 0) {
-                tupleCount--;
+            if (endOffset <= 0) {
+                indexCount--;
             } else {
                 break;
             }
         }
-        setTupleCount(tupleCount);
+        setIndexCount(indexCount);
+        if (nextIndex > indexCount) {
+            setNextIndex(indexCount);
+        }
     }
 
     @Override
@@ -178,7 +313,8 @@
 
     @Override
     public int getContiguousFreeSpace() {
-        return getTupleCountOffset() - tupleCount * 4 - freeDataEndOffset;
+        int slotSpace = indexCount * SIZE_OFFSET_GROUP;
+        return indexSlotsOffset - tupleAppend - slotSpace;
     }
 
     @Override
@@ -202,6 +338,11 @@
     }
 
     @Override
+    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
+        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+    }
+
+    @Override
     public int getFieldLength(int tupleIndex, int fIdx) {
         return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
     }
@@ -209,35 +350,58 @@
     @Override
     public int getTupleLength(int tupleIndex) {
         int endOffset = getTupleEndOffset(tupleIndex);
-        if (endOffset < 0) {
+        if (endOffset > 0) {
+            return endOffset - getTupleStartOffset(tupleIndex);
+        } else {
             return endOffset + getTupleStartOffset(tupleIndex);
         }
-        return endOffset - getTupleStartOffset(tupleIndex);
+    }
+
+    private void setTupleOffsets(int tupleIndex, int start, int length) {
+        setTupleStartOffset(tupleIndex, start);
+        setTupleEndOffset(tupleIndex, start + length);
     }
 
     @Override
     public int getTupleEndOffset(int tupleIndex) {
-        return IntSerDeUtils.getInt(array, tupleCountOffset - 4 * (tupleIndex + 1));
+        return IntSerDeUtils.getInt(array, getTupleEndSlotOffset(tupleIndex));
     }
 
     private void setTupleEndOffset(int tupleIndex, int offset) {
-        IntSerDeUtils.putInt(array, tupleCountOffset - 4 * (tupleIndex + 1), offset);
+        IntSerDeUtils.putInt(array, getTupleEndSlotOffset(tupleIndex), offset);
     }
 
     @Override
     public int getTupleStartOffset(int tupleIndex) {
-        int offset = tupleIndex == 0 ? 0 : IntSerDeUtils.getInt(array, tupleCountOffset - 4 * tupleIndex);
-        return Math.abs(offset);
+        return IntSerDeUtils.getInt(array, getTupleStartSlotOffset(tupleIndex));
     }
 
-    @Override
-    public int getAbsoluteFieldStartOffset(int tupleIndex, int fIdx) {
-        return getTupleStartOffset(tupleIndex) + getFieldSlotsLength() + getFieldStartOffset(tupleIndex, fIdx);
+    public void setTupleStartOffset(int tupleIndex, int offset) {
+        IntSerDeUtils.putInt(array, getTupleStartSlotOffset(tupleIndex), offset);
+    }
+
+    public int getTupleStartSlotOffset(int tupleIndex) {
+        return indexSlotsOffset - SIZE_OFFSET_GROUP * tupleIndex - SIZE_START_OFFSET;
+    }
+
+    public int getTupleEndSlotOffset(int tupleIndex) {
+        return getTupleStartSlotOffset(tupleIndex) - SIZE_END_OFFSET;
     }
 
     @Override
     public int getTupleCount() {
-        return tupleCount;
+        return indexCount;
+    }
+
+    private int getLiveTupleCount() {
+        int live = 0;
+        for (int i = 0; i < indexCount; ++i) {
+            int endOffset = getTupleEndOffset(i);
+            if (endOffset > 0) {
+                live++;
+            }
+        }
+        return live;
     }
 
     @Override
@@ -245,4 +409,12 @@
         return buffer;
     }
 
+    @Override
+    public void printStats(PrintStream ps) {
+        if (getLiveTupleCount() == 0) {
+            ps.print("");
+        }
+        ps.printf("(%d, %d)", getLiveTupleCount(), getIndexCount());
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
index 31ea07d..e7d1ceb 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IAppendDeletableFrameTupleAccessor.java
@@ -19,6 +19,7 @@
 
 package org.apache.hyracks.dataflow.std.sort.util;
 
+import java.io.PrintStream;
 import java.nio.ByteBuffer;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
@@ -73,4 +74,6 @@
      * @return how many contiguous free space in the buffer.
      */
     int getContiguousFreeSpace();
+
+    void printStats(PrintStream ps);
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.java
new file mode 100644
index 0000000..99f327b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPair.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.dataflow.std.sort.util;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+public class IntegerPair implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    public static final Comparator<IntegerPair> LEFT_ASC_COMPARATOR = new Comparator<IntegerPair>() {
+        @Override
+        public int compare(IntegerPair p1, IntegerPair p2) {
+            return p1.getLeft() - p2.getLeft();
+        }
+
+    };
+
+    public static final Comparator<IntegerPair> RIGHT_ASC_COMPARATOR = new Comparator<IntegerPair>() {
+        @Override
+        public int compare(IntegerPair p1, IntegerPair p2) {
+            return p1.getRight() - p2.getRight();
+        }
+
+    };
+
+    private int left;
+    private int right;
+
+    public IntegerPair() {
+        reset(Integer.MIN_VALUE, Integer.MIN_VALUE);
+    }
+
+    public IntegerPair(int l, int r) {
+        reset(l, r);
+    }
+
+    public int getLeft() {
+        return left;
+    }
+
+    public int getRight() {
+        return right;
+    }
+
+    public void reset(int l, int r) {
+        left = l;
+        right = r;
+    }
+
+    @Override
+    public String toString() {
+        return left + "," + right;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof IntegerPair)) {
+            return false;
+        } else {
+            IntegerPair p = (IntegerPair) obj;
+            return this.left == p.getLeft() && this.right == p.getRight();
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return left * 31 + right;
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java
new file mode 100644
index 0000000..330e822
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/IntegerPairPool.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hyracks.dataflow.std.sort.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class IntegerPairPool {
+    private final List<IntegerPair> list;
+
+    public IntegerPairPool() {
+        list = new ArrayList<>();
+    }
+
+    public IntegerPair takeOne() {
+        if (list.isEmpty()) {
+            return new IntegerPair();
+        }
+        return list.remove(list.size() - 1);
+    }
+
+    public void giveBack(IntegerPair pair) {
+        list.add(pair);
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/RunFilePointer.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/RunFilePointer.java
new file mode 100644
index 0000000..0c4e6c1
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/RunFilePointer.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.hyracks.dataflow.std.structures;
+
+import java.util.Comparator;
+
+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 = new Comparator<RunFilePointer>() {
+        @Override
+        public int compare(RunFilePointer tp1, RunFilePointer tp2) {
+            int c = (int) (tp1.getFileOffset() - tp2.getFileOffset());
+            if (c == 0) {
+                c = tp1.getTupleIndex() - tp2.getTupleIndex();
+            }
+            return c;
+        }
+
+    };
+
+    public static final Comparator<RunFilePointer> DESC = new Comparator<RunFilePointer>() {
+        @Override
+        public int compare(RunFilePointer tp1, RunFilePointer 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/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
index ae8eb24..42351a7 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
@@ -18,11 +18,37 @@
  */
 package org.apache.hyracks.dataflow.std.structures;
 
+import java.util.Comparator;
+
 public final class TuplePointer implements IResetable<TuplePointer> {
     public static final int INVALID_ID = -1;
     private int frameIndex;
     private int tupleIndex;
 
+    public static final Comparator<TuplePointer> ASC = new Comparator<TuplePointer>() {
+        @Override
+        public int compare(TuplePointer tp1, TuplePointer tp2) {
+            int c = tp1.getFrameIndex() - tp2.getFrameIndex();
+            if (c == 0) {
+                c = tp1.getTupleIndex() - tp2.getTupleIndex();
+            }
+            return c;
+        }
+
+    };
+
+    public static final Comparator<TuplePointer> DESC = new Comparator<TuplePointer>() {
+        @Override
+        public int compare(TuplePointer tp1, TuplePointer tp2) {
+            int c = tp2.getFrameIndex() - tp1.getFrameIndex();
+            if (c == 0) {
+                c = tp2.getTupleIndex() - tp1.getTupleIndex();
+            }
+            return c;
+        }
+
+    };
+
     public TuplePointer() {
         this(INVALID_ID, INVALID_ID);
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
index c58a2fa..e76928b 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
@@ -47,6 +47,8 @@
     FrameTupleAccessor inFTA = new FrameTupleAccessor(recordDescriptor);
     Random random = new Random(System.currentTimeMillis());
 
+    abstract ITupleAccessor getTupleAccessor();
+
     abstract ITuplePointerAccessor getTuplePointerAccessor();
 
     protected void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
index 7cc6762..3ee6425 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
@@ -150,4 +150,9 @@
     ITuplePointerAccessor getTuplePointerAccessor() {
         return bufferManager.getTuplePointerAccessor(recordDescriptor);
     }
+
+    @Override
+    ITupleAccessor getTupleAccessor() {
+        return bufferManager.getTupleAccessor(recordDescriptor);
+    }
 }
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
index e2a231f..a827d63 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
@@ -112,6 +112,11 @@
         return tupleMemoryManager.createTuplePointerAccessor();
     }
 
+    @Override
+    ITupleAccessor getTupleAccessor() {
+        return tupleMemoryManager.createTupleAccessor();
+    }
+
     private Map<TuplePointer, Integer> insertInFTAToBufferShouldAllSuccess() throws HyracksDataException {
         Map<TuplePointer, Integer> tuplePointerIntegerMap = new HashMap<>();
         for (int i = 0; i < inFTA.getTupleCount(); i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
index 7686540..f8edce6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppenderTest.java
@@ -31,23 +31,27 @@
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
 import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import org.apache.hyracks.util.IntSerDeUtils;
 import org.apache.hyracks.dataflow.std.sort.Utility;
+import org.apache.hyracks.util.IntSerDeUtils;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 import org.junit.Before;
 import org.junit.Test;
 
+/**
+ * @see org.apache.hyracks.dataflow.std.sort.util.DeletableFrameTupleAppender
+ */
 public class DeletableFrameTupleAppenderTest {
+    private static final int META_DATA_SIZE = 4 + 4 + 4 + 4;
+    private static final int SLOT_SIZE = 4 + 4;
+    private static final char TEST_CH = 'x';
+    private static final int TEST_TUPLE_COUNT = 8;
+    private static final int TEST_FRAME_SIZE = 256;
+
     DeletableFrameTupleAppender appender;
-    ISerializerDeserializer[] fields = new ISerializerDeserializer[] {
-            IntegerSerializerDeserializer.INSTANCE,
-            new UTF8StringSerializerDeserializer(),
-    };
+    ISerializerDeserializer[] fields = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+            new UTF8StringSerializerDeserializer(), };
     RecordDescriptor recordDescriptor = new RecordDescriptor(fields);
     ArrayTupleBuilder builder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
-    static final char TEST_CH = 'x';
-
-    int cap = 256;
 
     @Before
     public void initial() throws HyracksDataException {
@@ -56,30 +60,46 @@
 
     @Test
     public void testClear() throws Exception {
-        ByteBuffer buffer = ByteBuffer.allocate(cap);
+        ByteBuffer buffer = ByteBuffer.allocate(TEST_FRAME_SIZE);
         appender.clear(buffer);
         assertTrue(appender.getBuffer() == buffer);
         assertTrue(appender.getTupleCount() == 0);
-        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+        assertTrue(appender.getTotalFreeSpace() == TEST_FRAME_SIZE - META_DATA_SIZE);
+        assertTrue(appender.getContiguousFreeSpace() == TEST_FRAME_SIZE - META_DATA_SIZE);
     }
 
     ByteBuffer makeAFrame(int capacity, int count, int deletedBytes) throws HyracksDataException {
         ByteBuffer buffer = ByteBuffer.allocate(capacity);
         int metaOffset = capacity - 4;
-        buffer.putInt(metaOffset, deletedBytes);
-        metaOffset -= 4;
         buffer.putInt(metaOffset, count);
         metaOffset -= 4;
+        buffer.putInt(metaOffset, deletedBytes);
+        // next index
+        metaOffset -= 4;
+        buffer.putInt(metaOffset, count);
+        // append slot
+        metaOffset -= 4;
+        int appendOffset = metaOffset;
+        buffer.putInt(metaOffset, 0);
+        metaOffset -= 4;
+
+        int start = 0;
         for (int i = 0; i < count; i++, metaOffset -= 4) {
             makeARecord(builder, i);
             for (int x = 0; x < builder.getFieldEndOffsets().length; x++) {
                 buffer.putInt(builder.getFieldEndOffsets()[x]);
             }
             buffer.put(builder.getByteArray(), 0, builder.getSize());
-            assert (metaOffset > buffer.position());
+
+            // Add slot information
+            buffer.putInt(metaOffset, start);
+            metaOffset -= 4;
             buffer.putInt(metaOffset, buffer.position());
 
+            start = buffer.position();
+            assert (metaOffset > buffer.position());
         }
+        buffer.putInt(appendOffset, start);
         return buffer;
     }
 
@@ -89,52 +109,52 @@
         builder.addField(fields[1], Utility.repeatString(TEST_CH, i + 1));
     }
 
-    int assertTupleIsExpected(int i, int dataOffset) {
-        int lenStrMeta = UTF8StringUtil.getNumBytesToStoreLength(i);
-        int tupleLength = 2 * 4 + 4 + lenStrMeta + i + 1;
+    int assertTupleIsExpected(int i, int dataOffset, int testString) {
+        int lenStrMeta = UTF8StringUtil.getNumBytesToStoreLength(testString);
+        int tupleLength = 2 * 4 + 4 + lenStrMeta + testString + 1;
         assertEquals(dataOffset, appender.getTupleStartOffset(i));
         assertEquals(tupleLength, appender.getTupleLength(i));
 
         assertEquals(dataOffset + 2 * 4, appender.getAbsoluteFieldStartOffset(i, 0));
         assertEquals(4, appender.getFieldLength(i, 0));
-        assertEquals(i + 1,
+        assertEquals(testString + 1,
                 IntSerDeUtils.getInt(appender.getBuffer().array(), appender.getAbsoluteFieldStartOffset(i, 0)));
         assertEquals(dataOffset + 2 * 4 + 4, appender.getAbsoluteFieldStartOffset(i, 1));
-        assertEquals(lenStrMeta + i + 1, appender.getFieldLength(i, 1));
+        assertEquals(lenStrMeta + testString + 1, appender.getFieldLength(i, 1));
         return tupleLength;
     }
 
     @Test
     public void testReset() throws Exception {
-        ByteBuffer buffer = ByteBuffer.allocate(cap);
+        ByteBuffer buffer = ByteBuffer.allocate(TEST_FRAME_SIZE);
         appender.reset(buffer);
         assertTrue(appender.getBuffer() == buffer);
         assertTrue(appender.getTupleCount() == 0);
-        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4);
+        assertTrue(appender.getContiguousFreeSpace() == TEST_FRAME_SIZE - META_DATA_SIZE);
 
-        int count = 10;
+        int count = TEST_TUPLE_COUNT;
         int deleted = 7;
-        buffer = makeAFrame(cap, count, deleted);
+        buffer = makeAFrame(TEST_FRAME_SIZE, count, deleted);
         int pos = buffer.position();
         appender.reset(buffer);
         assertTrue(appender.getBuffer() == buffer);
         assertTrue(appender.getTupleCount() == count);
-        assertTrue(appender.getContiguousFreeSpace() == cap - 4 - 4 - count * 4 - pos);
+        assertTrue(appender.getContiguousFreeSpace() == TEST_FRAME_SIZE - META_DATA_SIZE - count * SLOT_SIZE - pos);
         assertTrue(appender.getTotalFreeSpace() == appender.getContiguousFreeSpace() + deleted);
 
         int dataOffset = 0;
         for (int i = 0; i < count; i++) {
-            dataOffset += assertTupleIsExpected(i, dataOffset);
+            dataOffset += assertTupleIsExpected(i, dataOffset, i);
         }
     }
 
     @Test
     public void testAppend() throws Exception {
-        int count = 10;
-        ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+        int count = TEST_TUPLE_COUNT;
+        ByteBuffer bufferRead = makeAFrame(TEST_FRAME_SIZE, count, 0);
         DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
         accessor.reset(bufferRead);
-        ByteBuffer bufferWrite = ByteBuffer.allocate(cap);
+        ByteBuffer bufferWrite = ByteBuffer.allocate(TEST_FRAME_SIZE);
         appender.clear(bufferWrite);
         for (int i = 0; i < accessor.getTupleCount(); i++) {
             appender.append(accessor, i);
@@ -146,9 +166,9 @@
 
     @Test
     public void testDelete() throws Exception {
-        int count = 10;
+        int count = TEST_TUPLE_COUNT;
         int deleteSpace = 0;
-        ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
+        ByteBuffer buffer = makeAFrame(TEST_FRAME_SIZE, count, deleteSpace);
         appender.reset(buffer);
 
         int freeSpace = appender.getContiguousFreeSpace();
@@ -156,7 +176,7 @@
             deleteSpace += assertDeleteSucceed(i, freeSpace, deleteSpace);
             int innerOffset = deleteSpace;
             for (int j = i + 1; j < appender.getTupleCount(); j++) {
-                innerOffset += assertTupleIsExpected(j, innerOffset);
+                innerOffset += assertTupleIsExpected(j, innerOffset, j);
             }
         }
     }
@@ -165,7 +185,8 @@
     public void testResetAfterDelete() throws Exception {
         testDelete();
         appender.reset(appender.getBuffer());
-        assertEquals(cap - appender.getTupleCount() * 4 - 4 - 4, appender.getTotalFreeSpace());
+        assertEquals(TEST_FRAME_SIZE - appender.getTupleCount() * SLOT_SIZE - META_DATA_SIZE,
+                appender.getTotalFreeSpace());
 
     }
 
@@ -187,14 +208,14 @@
     @Test
     public void testAppendAndDelete() throws Exception {
         int cap = 1024;
-        int count = 10;
+        int count = TEST_TUPLE_COUNT;
         int deleteSpace = 0;
         ByteBuffer buffer = makeAFrame(cap, count, deleteSpace);
         int dataOffset = buffer.position();
         appender.reset(buffer);
 
         int freeSpace = appender.getContiguousFreeSpace();
-        int[] deleteSet = new int[] { 1, 3, 5 };
+        int[] deleteSet = new int[] { 1, 3, 5, 7 };
         for (int i = 0; i < deleteSet.length; i++) {
             deleteSpace += assertDeleteSucceed(deleteSet[i], freeSpace, deleteSpace);
         }
@@ -203,28 +224,28 @@
         DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
         accessor.reset(bufferRead);
 
+        int[] appendSet = new int[] { 1, 3, 5, 7, 8, 9, 10, 11 };
         for (int i = count; i < accessor.getTupleCount(); i++) {
             int id = appender.append(accessor, i);
-            dataOffset += assertTupleIsExpected(i, dataOffset);
-            assertEquals(i, id);
+            dataOffset += assertTupleIsExpected(id, dataOffset, i);
+            assertEquals(appendSet[i - count], id);
         }
 
         appender.reOrganizeBuffer();
         dataOffset = 0;
+        int[] appendOrder = new int[] { 0, 2, 4, 6, 1, 3, 5, 7, 8, 9, 10, 11 };
+        int[] stringSize = new int[] { 0, 2, 4, 6, 8, 9, 10, 11, 12, 13, 14, 15 };
         for (int i = 0; i < appender.getTupleCount(); i++) {
-            if (ArrayUtils.contains(deleteSet, i)) {
-                continue;
-            }
-            dataOffset += assertTupleIsExpected(i, dataOffset);
+            dataOffset += assertTupleIsExpected(appendOrder[i], dataOffset, stringSize[i]);
         }
     }
 
     @Test
     public void testReOrganizeBuffer() throws Exception {
-        int count = 10;
+        int count = TEST_TUPLE_COUNT;
         testDelete();
         appender.reOrganizeBuffer();
-        ByteBuffer bufferRead = makeAFrame(cap, count, 0);
+        ByteBuffer bufferRead = makeAFrame(TEST_FRAME_SIZE, count, 0);
         DeletableFrameTupleAppender accessor = new DeletableFrameTupleAppender(recordDescriptor);
         accessor.reset(bufferRead);
         for (int i = 0; i < accessor.getTupleCount(); i++) {
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 1a5d4ab..3692cd5 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -91,6 +91,11 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-dataflow-common</artifactId>
+      <version>0.2.18-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-client</artifactId>
       <version>${project.version}</version>
     </dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/FieldRangePartitionComputerFactoryTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/FieldRangePartitionComputerFactoryTest.java
new file mode 100644
index 0000000..637e195
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/FieldRangePartitionComputerFactoryTest.java
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.tests.unit;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryRangeComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRangeMap;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITupleRangePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.dataflow.value.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.storage.IGrowableIntArray;
+import org.apache.hyracks.data.std.accessors.PointableBinaryRangeAscComparatorFactory;
+import org.apache.hyracks.data.std.accessors.PointableBinaryRangeDescComparatorFactory;
+import org.apache.hyracks.data.std.api.IComparable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IPointableFactory;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.dataflow.common.comm.io.FrameFixedFieldTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.data.marshalling.Integer64SerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.storage.common.arraylist.IntArrayList;
+import org.apache.hyracks.test.support.TestUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class FieldRangePartitionComputerFactoryTest extends TestCase {
+
+    private final Integer64SerializerDeserializer int64Serde = Integer64SerializerDeserializer.INSTANCE;
+    @SuppressWarnings("rawtypes")
+    private final ISerializerDeserializer[] SerDers = new ISerializerDeserializer[] {
+            Integer64SerializerDeserializer.INSTANCE };
+    private final RecordDescriptor RecordDesc = new RecordDescriptor(SerDers);
+
+    IBinaryRangeComparatorFactory[] BINARY_ASC_COMPARATOR_FACTORIES = new IBinaryRangeComparatorFactory[] {
+            new PointableBinaryRangeAscComparatorFactory(LongPointable.FACTORY) };
+    IBinaryRangeComparatorFactory[] BINARY_DESC_COMPARATOR_FACTORIES = new IBinaryRangeComparatorFactory[] {
+            new PointableBinaryRangeDescComparatorFactory(LongPointable.FACTORY) };
+    IBinaryRangeComparatorFactory[] BINARY_REPLICATE_COMPARATOR_FACTORIES = new IBinaryRangeComparatorFactory[] {
+            new PointableBinaryReplicateRangeComparatorFactory(LongPointable.FACTORY) };
+    /*
+     * The following points (X) will be tested for these 4 partitions.
+     *
+     * X-------X----XXX----X----XXX----X----XXX----X-------X
+     *    -----------|-----------|-----------|-----------
+     *
+     * The following points (X) will be tested for these 16 partitions.
+     *
+     * X-------X----XXX----X----XXX----X----XXX----X-------X
+     *    --|--|--|--|--|--|--|--|--|--|--|--|--|--|--|--
+     */
+
+    private final int FRAME_SIZE = 320;
+    private final int INTEGER_LENGTH = Long.BYTES;
+
+    //result index {      0,   1,   2,   3,    4,    5,    6,    7,    8,    9,   10,   11,   12,   13,   14,   15   };
+    //points       {     20l, 45l, 70l, 95l, 120l, 145l, 170l, 195l, 220l, 245l, 270l, 295l, 320l, 345l, 370l, 395l  };
+    private final Long[] EACH_PARTITION = new Long[] { 20l, 45l, 70l, 95l, 120l, 145l, 170l, 195l, 220l, 245l, 270l,
+            295l, 320l, 345l, 370l, 395l };
+
+    //result index {      0,   1,   2,   3,    4,    5,    6,    7,    8,    9,    10,   11,   12,   13,   14        };
+    //points       {    -25l, 50l, 99l, 100l, 101l, 150l, 199l, 200l, 201l, 250l, 299l, 300l, 301l, 350l, 425l       };
+    private final Long[] PARTITION_BOUNDRIES = new Long[] { -25l, 50l, 99l, 100l, 101l, 150l, 199l, 200l, 201l, 250l,
+            299l, 300l, 301l, 350l, 425l };
+
+    //map          { 0l, 25l, 50l, 75l, 100l, 125l, 150l, 175l, 200l, 225l, 250l, 275l, 300l, 325l, 350l, 375l, 400l };
+    //partitions   {    0,   1,   2,   3,    4,    5,    6,    7,    8,    9,    10,   11,   12,   13,   14,  15     };
+    private final Long[] MAP_POINTS = new Long[] { 0l, 25l, 50l, 75l, 100l, 125l, 150l, 175l, 200l, 225l, 250l, 275l,
+            300l, 325l, 350l, 375l, 400l };
+
+    private byte[] getIntegerBytes(Long[] integers) throws HyracksDataException {
+        try {
+            ByteArrayOutputStream bos = new ByteArrayOutputStream();
+            DataOutput dos = new DataOutputStream(bos);
+            for (int i = 0; i < integers.length; ++i) {
+                int64Serde.serialize(integers[i], dos);
+            }
+            bos.close();
+            return bos.toByteArray();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    private IRangeMap getRangeMap(Long[] integers) throws HyracksDataException {
+        int offsets[] = new int[integers.length];
+        for (int i = 0; i < integers.length; ++i) {
+            offsets[i] = (i + 1) * INTEGER_LENGTH;
+        }
+        return new RangeMap(1, getIntegerBytes(integers), offsets);
+    }
+
+    private ByteBuffer prepareData(IHyracksTaskContext ctx, Long[] integers) throws HyracksDataException {
+        IFrame frame = new VSizeFrame(ctx);
+        FrameFixedFieldTupleAppender fffta = new FrameFixedFieldTupleAppender(RecordDesc.getFieldCount());
+        fffta.reset(frame, true);
+
+        byte[] serializedIntegers = getIntegerBytes(integers);
+        for (int i = 0; i < integers.length; ++i) {
+            fffta.appendField(serializedIntegers, i * INTEGER_LENGTH, INTEGER_LENGTH);
+        }
+
+        return frame.getBuffer();
+    }
+
+    private void executeFieldRangePartitionTests(Long[] integers, IRangeMap rangeMap,
+            IBinaryRangeComparatorFactory[] comparatorFactories, RangePartitioningType rangeType, int nParts,
+            int[][] results) throws HyracksDataException {
+        IHyracksTaskContext ctx = TestUtils.create(FRAME_SIZE);
+        int[] rangeFields = new int[] { 0 };
+        ITupleRangePartitionComputerFactory frpcf = new FieldRangePartitionComputerFactory(rangeFields,
+                comparatorFactories, rangeType);
+        ITupleRangePartitionComputer partitioner = frpcf.createPartitioner(rangeMap);
+
+        IFrameTupleAccessor accessor = new FrameTupleAccessor(RecordDesc);
+        ByteBuffer buffer = prepareData(ctx, integers);
+        accessor.reset(buffer);
+
+        IGrowableIntArray map = new IntArrayList(16, 1);
+
+        for (int i = 0; i < results.length; ++i) {
+            map.clear();
+            partitioner.partition(accessor, i, nParts, map);
+            checkPartitionResult(integers[i], results[i], map);
+        }
+    }
+
+    private String getString(int[] results) {
+        String result = "[";
+        for (int i = 0; i < results.length; ++i) {
+            result += results[i];
+            if (i < results.length - 1) {
+                result += ", ";
+            }
+        }
+        result += "]";
+        return result;
+    }
+
+    private String getString(IGrowableIntArray results) {
+        String result = "[";
+        for (int i = 0; i < results.size(); ++i) {
+            result += results.get(i);
+            if (i < results.size() - 1) {
+                result += ", ";
+            }
+        }
+        result += "]";
+        return result;
+    }
+
+    private void checkPartitionResult(Long value, int[] results, IGrowableIntArray map) {
+        if (results.length != map.size()) {
+            Assert.assertEquals("The partition for value (" + value + ") gives different number of partitions",
+                    results.length, map.size());
+        }
+        for (int i = 0; i < results.length; ++i) {
+            boolean match = false;
+            for (int j = 0; j < results.length; ++j) {
+                if (results[i] == map.get(j)) {
+                    match = true;
+                    continue;
+                }
+            }
+            if (!match) {
+                Assert.assertEquals("Individual partitions for " + value + " do not match", getString(results),
+                        getString(map));
+                return;
+            }
+        }
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProject4AllPartitions() throws HyracksDataException {
+        int[][] results = new int[16][];
+        results[0] = new int[] { 0 };
+        results[1] = new int[] { 0 };
+        results[2] = new int[] { 0 };
+        results[3] = new int[] { 0 };
+        results[4] = new int[] { 1 };
+        results[5] = new int[] { 1 };
+        results[6] = new int[] { 1 };
+        results[7] = new int[] { 1 };
+        results[8] = new int[] { 2 };
+        results[9] = new int[] { 2 };
+        results[10] = new int[] { 2 };
+        results[11] = new int[] { 2 };
+        results[12] = new int[] { 3 };
+        results[13] = new int[] { 3 };
+        results[14] = new int[] { 3 };
+        results[15] = new int[] { 3 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(EACH_PARTITION, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 4, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescProject4AllPartitions() throws HyracksDataException {
+        int[][] results = new int[16][];
+        results[0] = new int[] { 3 };
+        results[1] = new int[] { 3 };
+        results[2] = new int[] { 3 };
+        results[3] = new int[] { 3 };
+        results[4] = new int[] { 2 };
+        results[5] = new int[] { 2 };
+        results[6] = new int[] { 2 };
+        results[7] = new int[] { 2 };
+        results[8] = new int[] { 1 };
+        results[9] = new int[] { 1 };
+        results[10] = new int[] { 1 };
+        results[11] = new int[] { 1 };
+        results[12] = new int[] { 0 };
+        results[13] = new int[] { 0 };
+        results[14] = new int[] { 0 };
+        results[15] = new int[] { 0 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(EACH_PARTITION, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 4, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProject16AllPartitions() throws HyracksDataException {
+        int[][] results = new int[16][];
+        results[0] = new int[] { 0 };
+        results[1] = new int[] { 1 };
+        results[2] = new int[] { 2 };
+        results[3] = new int[] { 3 };
+        results[4] = new int[] { 4 };
+        results[5] = new int[] { 5 };
+        results[6] = new int[] { 6 };
+        results[7] = new int[] { 7 };
+        results[8] = new int[] { 8 };
+        results[9] = new int[] { 9 };
+        results[10] = new int[] { 10 };
+        results[11] = new int[] { 11 };
+        results[12] = new int[] { 12 };
+        results[13] = new int[] { 13 };
+        results[14] = new int[] { 14 };
+        results[15] = new int[] { 15 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(EACH_PARTITION, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionDescProject16AllPartitions() throws HyracksDataException {
+        int[][] results = new int[16][];
+        results[0] = new int[] { 15 };
+        results[1] = new int[] { 14 };
+        results[2] = new int[] { 13 };
+        results[3] = new int[] { 12 };
+        results[4] = new int[] { 11 };
+        results[5] = new int[] { 10 };
+        results[6] = new int[] { 9 };
+        results[7] = new int[] { 8 };
+        results[8] = new int[] { 7 };
+        results[9] = new int[] { 6 };
+        results[10] = new int[] { 5 };
+        results[11] = new int[] { 4 };
+        results[12] = new int[] { 3 };
+        results[13] = new int[] { 2 };
+        results[14] = new int[] { 1 };
+        results[15] = new int[] { 0 };
+
+        Long[] map = MAP_POINTS.clone();
+        ArrayUtils.reverse(map);
+        IRangeMap rangeMap = getRangeMap(map);
+
+        executeFieldRangePartitionTests(EACH_PARTITION, rangeMap, BINARY_DESC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProject16Partitions() throws HyracksDataException {
+        int[][] results = new int[15][];
+        results[0] = new int[] { 0 };
+        results[1] = new int[] { 2 };
+        results[2] = new int[] { 3 };
+        results[3] = new int[] { 4 };
+        results[4] = new int[] { 4 };
+        results[5] = new int[] { 6 };
+        results[6] = new int[] { 7 };
+        results[7] = new int[] { 8 };
+        results[8] = new int[] { 8 };
+        results[9] = new int[] { 10 };
+        results[10] = new int[] { 11 };
+        results[11] = new int[] { 12 };
+        results[12] = new int[] { 12 };
+        results[13] = new int[] { 14 };
+        results[14] = new int[] { 15 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(PARTITION_BOUNDRIES, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 16, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscProject4Partitions() throws HyracksDataException {
+        int[][] results = new int[15][];
+        results[0] = new int[] { 0 };
+        results[1] = new int[] { 0 };
+        results[2] = new int[] { 0 };
+        results[3] = new int[] { 1 };
+        results[4] = new int[] { 1 };
+        results[5] = new int[] { 1 };
+        results[6] = new int[] { 1 };
+        results[7] = new int[] { 2 };
+        results[8] = new int[] { 2 };
+        results[9] = new int[] { 2 };
+        results[10] = new int[] { 2 };
+        results[11] = new int[] { 3 };
+        results[12] = new int[] { 3 };
+        results[13] = new int[] { 3 };
+        results[14] = new int[] { 3 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(PARTITION_BOUNDRIES, rangeMap, BINARY_ASC_COMPARATOR_FACTORIES,
+                RangePartitioningType.PROJECT, 4, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscReplicate4Partitions() throws HyracksDataException {
+        int[][] results = new int[15][];
+        results[0] = new int[] { 0, 1, 2, 3 };
+        results[1] = new int[] { 0, 1, 2, 3 };
+        results[2] = new int[] { 0, 1, 2, 3 };
+        results[3] = new int[] { 1, 2, 3 };
+        results[4] = new int[] { 1, 2, 3 };
+        results[5] = new int[] { 1, 2, 3 };
+        results[6] = new int[] { 1, 2, 3 };
+        results[7] = new int[] { 2, 3 };
+        results[8] = new int[] { 2, 3 };
+        results[9] = new int[] { 2, 3 };
+        results[10] = new int[] { 2, 3 };
+        results[11] = new int[] { 3 };
+        results[12] = new int[] { 3 };
+        results[13] = new int[] { 3 };
+        results[14] = new int[] { 3 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(PARTITION_BOUNDRIES, rangeMap, BINARY_REPLICATE_COMPARATOR_FACTORIES,
+                RangePartitioningType.REPLICATE, 4, results);
+    }
+
+    @Test
+    public void testFieldRangePartitionAscReplicate16Partitions() throws HyracksDataException {
+        int[][] results = new int[15][];
+        results[0] = new int[] { 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[1] = new int[] { 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[2] = new int[] { 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[3] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[4] = new int[] { 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[5] = new int[] { 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[6] = new int[] { 7, 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[7] = new int[] { 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[8] = new int[] { 8, 9, 10, 11, 12, 13, 14, 15 };
+        results[9] = new int[] { 10, 11, 12, 13, 14, 15 };
+        results[10] = new int[] { 11, 12, 13, 14, 15 };
+        results[11] = new int[] { 12, 13, 14, 15 };
+        results[12] = new int[] { 12, 13, 14, 15 };
+        results[13] = new int[] { 14, 15 };
+        results[14] = new int[] { 15 };
+
+        IRangeMap rangeMap = getRangeMap(MAP_POINTS);
+
+        executeFieldRangePartitionTests(PARTITION_BOUNDRIES, rangeMap, BINARY_REPLICATE_COMPARATOR_FACTORIES,
+                RangePartitioningType.REPLICATE, 16, results);
+    }
+
+    private class PointableBinaryReplicateRangeComparatorFactory implements IBinaryRangeComparatorFactory {
+        private static final long serialVersionUID = 1L;
+
+        protected final IPointableFactory pf;
+
+        public PointableBinaryReplicateRangeComparatorFactory(IPointableFactory pf) {
+            this.pf = pf;
+        }
+
+        @Override
+        public IBinaryComparator createMinBinaryComparator() {
+            final IPointable p = pf.createPointable();
+            return new IBinaryComparator() {
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                    if (l1 == 0 && l2 != 0)
+                        return -1;
+                    if (l1 != 0 && l2 == 0)
+                        return 1;
+                    p.set(b1, s1, l1);
+                    return ((IComparable) p).compareTo(b2, s2, l2);
+                }
+            };
+        }
+
+        @Override
+        public IBinaryComparator createMaxBinaryComparator() {
+            return new IBinaryComparator() {
+                @Override
+                public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                    return -1;
+                }
+            };
+        }
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
index e164b6b..1a7f72d 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
@@ -19,7 +19,9 @@
 
 package org.apache.hyracks.storage.common.arraylist;
 
-public class IntArrayList {
+import org.apache.hyracks.api.storage.IGrowableIntArray;
+
+public class IntArrayList implements IGrowableIntArray {
     private int[] data;
     private int size;
     private int first;
@@ -32,6 +34,7 @@
         this.growth = growth;
     }
 
+    @Override
     public int size() {
         return size;
     }
@@ -40,6 +43,7 @@
         return first;
     }
 
+    @Override
     public void add(int i) {
         if (size == data.length) {
             int[] newData = new int[data.length + growth];
@@ -70,6 +74,7 @@
         return data[size - 1];
     }
 
+    @Override
     public int get(int i) {
         return data[i];
     }
@@ -88,6 +93,7 @@
         first++;
     }
 
+    @Override
     public void clear() {
         size = 0;
         first = 0;