asterixdb-notifications mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Preston Carman (Code Review)" <do-not-re...@asterixdb.incubator.apache.org>
Subject Change in asterixdb[master]: Adding interval merge join.
Date Thu, 15 Oct 2015 23:57:10 GMT
Preston Carman has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/457

Change subject: Adding interval merge join.
......................................................................

Adding interval merge join.

Change-Id: Id4f9be43eb0c500f04f99438c34e69503591641d
---
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
M asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
A asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java
M asterix-app/data/csv/sample_01.csv
A asterix-app/src/test/resources/runtimets/queries/temporal/TemporalQueries.xml
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
A asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.4.adm
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.adm
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm
A asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm
M asterix-app/src/test/resources/runtimets/testsuite.xml
M asterix-aql/src/main/java/org/apache/asterix/aql/util/RangeMapBuilder.java
M asterix-aql/src/main/javacc/AQL.jj
A asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java
M asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
A asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/AllenRelationsBinaryComparatorFactoryProvider.java
A asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/OverlapIntervalBinaryComparatorFactory.java
A asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalProjectBinaryComparatorFactory.java
A asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalReplicateBinaryComparatorFactory.java
A asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalSplitBinaryComparatorFactory.java
M asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
A asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
M asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
M asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
A asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java
A asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java
A asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
A asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java
A asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
M pom.xml
42 files changed, 1,326 insertions(+), 201 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/57/457/1

diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index a1c9d8c..940a814 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -289,8 +289,8 @@
         //Turned off the following rule for now not to change OptimizerTest results.
         //physicalRewritesAllLevels.add(new IntroduceTransactionCommitByAssignOpRule());
         physicalRewritesAllLevels.add(new ReplaceSinkOpWithCommitOpRule());
-        physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
+        physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
         physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
index 35bc3fd..7489df2 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceRandomPartitioningFeedComputationRule.java
@@ -35,7 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -78,7 +78,7 @@
             }
         };
 
-        exchangeOp.setPhysicalOperator(new RandomPartitionPOperator(domain));
+        exchangeOp.setPhysicalOperator(new RandomPartitionExchangePOperator(domain));
         op.getInputs().get(0).setValue(exchangeOp);
         exchangeOp.getInputs().add(new MutableObject<ILogicalOperator>(scanOp));
         ExecutionMode em = ((AbstractLogicalOperator) scanOp).getExecutionMode();
diff --git a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 353c9be..9feaa09 100644
--- a/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -33,6 +33,7 @@
 import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
 import org.apache.asterix.optimizer.rules.am.BTreeJobGenParams;
+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;
@@ -61,7 +62,6 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
 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/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java
new file mode 100644
index 0000000..a335ec4
--- /dev/null
+++ b/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/JoinUtils.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.apache.asterix.common.annotations.IntervalJoinExpressionAnnotation;
+import org.apache.asterix.dataflow.data.nontagged.comparators.allenrelations.AllenRelationsBinaryComparatorFactoryProvider;
+import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
+import org.apache.asterix.runtime.operators.joins.OverlappedByIntervalMergeJoinCheckerFactory;
+import org.apache.asterix.runtime.operators.joins.OverlapsIntervalMergeJoinCheckerFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+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;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator.JoinPartitioningType;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.MergeJoinPOperator;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+
+public class JoinUtils {
+
+    private static final Logger LOGGER = Logger.getLogger(JoinUtils.class.getName());
+
+    public static void setJoinAlgorithmAndExchangeAlgo(AbstractBinaryJoinOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        List<LogicalVariable> sideLeft = new LinkedList<LogicalVariable>();
+        List<LogicalVariable> sideRight = new LinkedList<LogicalVariable>();
+        List<LogicalVariable> varsLeft = op.getInputs().get(0).getValue().getSchema();
+        List<LogicalVariable> varsRight = op.getInputs().get(1).getValue().getSchema();
+        ILogicalExpression conditionLE = op.getCondition().getValue();
+        if (conditionLE.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return;
+        }
+        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.getRangeMap(), context);
+            } else if (ijea.isIopJoin()) {
+                // Overlapping Interval Partition.
+                LOGGER.fine("Interval Join - IOP");
+            } else if (ijea.isSpatialJoin()) {
+                // Spatial Partition.
+                LOGGER.fine("Interval Join - Spatial Partitioning");
+            }
+        }
+    }
+
+    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, IRangeMap rangeMap,
+            IOptimizationContext context) {
+        IBinaryComparatorFactoryProvider bcfp = (IBinaryComparatorFactoryProvider) AllenRelationsBinaryComparatorFactoryProvider.INSTANCE;
+        IMergeJoinCheckerFactory mjcf = new OverlapsIntervalMergeJoinCheckerFactory();
+        if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+            mjcf = new OverlappedByIntervalMergeJoinCheckerFactory();
+        }
+        op.setPhysicalOperator(new MergeJoinPOperator(op.getJoinKind(), JoinPartitioningType.BROADCAST,
+                context.getPhysicalOptimizationConfig().getMaxRecordsPerFrame(), sideLeft, sideRight, bcfp, mjcf,
+                rangeMap));
+    }
+
+    private static FunctionIdentifier isIntervalJoinCondition(ILogicalExpression e,
+            Collection<LogicalVariable> inLeftAll, Collection<LogicalVariable> inRightAll,
+            Collection<LogicalVariable> outLeftFields, Collection<LogicalVariable> outRightFields) {
+        FunctionIdentifier fiReturn = null;
+        boolean switchArguments = false;
+        switch (e.getExpressionTag()) {
+            case FUNCTION_CALL: {
+                AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e;
+                FunctionIdentifier fi = fexp.getFunctionIdentifier();
+                if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPS)
+                        || fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+                    fiReturn = fi;
+                } else {
+                    return null;
+                }
+                ILogicalExpression opLeft = fexp.getArguments().get(0).getValue();
+                ILogicalExpression opRight = fexp.getArguments().get(1).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;
+            }
+            default: {
+                return null;
+            }
+        }
+    }
+
+    private static FunctionIdentifier reverseIntervalExpression(FunctionIdentifier fi) {
+        if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPS)) {
+            return AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY;
+        } else if (fi.equals(AsterixBuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
+            return AsterixBuiltinFunctions.INTERVAL_OVERLAPS;
+        }
+        return null;
+    }
+}
diff --git a/asterix-app/data/csv/sample_01.csv b/asterix-app/data/csv/sample_01.csv
index 4dd437a..fbba382 100644
--- a/asterix-app/data/csv/sample_01.csv
+++ b/asterix-app/data/csv/sample_01.csv
@@ -1,8 +1,8 @@
-1,0.899682764,5.6256,2013-08-07,07:22:35,1979-02-25T23:48:27.034
-2,0.669052398,,-1923-03-29,19:33:34,-1979-02-25T23:48:27.002
-3,0.572733058,192674,-1923-03-28,19:33:34,-1979-02-25T23:48:27.001
-4,,192674,-1923-03-27,19:33:34,-1979-02-25T23:48:27.001
-5,0.572733058,192674,,19:33:34,-1979-02-25T23:48:27.001
-6,0.572733058,192674,-1923-03-25,,-1979-02-25T23:48:27.001
-7,0.572733058,192674,-1923-03-24,19:33:34,
+1,0.899682764,5.6256,2013-08-07,07:22:35,1979-02-25T23:48:27.034
+2,0.669052398,,-1923-03-29,19:33:34,-1979-02-25T23:48:27.002
+3,0.572733058,192674,-1923-03-28,19:33:34,-1979-02-25T23:48:27.001
+4,,192674,-1923-03-27,19:33:34,-1979-02-25T23:48:27.001
+5,0.572733058,192674,,19:33:34,-1979-02-25T23:48:27.001
+6,0.572733058,192674,-1923-03-25,,-1979-02-25T23:48:27.001
+7,0.572733058,192674,-1923-03-24,19:33:34,
 8,,,,,
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/TemporalQueries.xml b/asterix-app/src/test/resources/runtimets/queries/temporal/TemporalQueries.xml
new file mode 100644
index 0000000..04a5329
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/TemporalQueries.xml
@@ -0,0 +1,156 @@
+<!--
+ ! Copyright 2009-2013 by The Regents of the University of California
+ ! Licensed under the Apache License, Version 2.0 (the "License");
+ ! you may not use this file except in compliance with the License.
+ ! you may obtain a copy of the License from
+ !
+ !     http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing, software
+ ! distributed under the License is distributed on an "AS IS" BASIS,
+ ! WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ ! See the License for the specific language governing permissions and
+ ! limitations under the License.
+ !-->
+        <test-case FilePath="temporal">
+            <compilation-unit name="overlap_bins_gby_3">
+                <output-dir compare="Text">overlap_bins_gby_3</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="agg_01">
+                <output-dir compare="Text">agg_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="overlap_bins_gby_1">
+                <output-dir compare="Text">overlap_bins_gby_1</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="duration_functions">
+                <output-dir compare="Text">duration_functions</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="overlap_bins_gby_0">
+                <output-dir compare="Text">overlap_bins_gby_0</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="get_overlapping_interval">
+                <output-dir compare="Text">get_overlapping_interval</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="overlap_bins">
+                <output-dir compare="Text">overlap_bins</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="parse_02">
+                <output-dir compare="Text">parse_02</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="parse_01">
+                <output-dir compare="Text">parse_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="day_of_week_01">
+                <output-dir compare="Text">day_of_week_01</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="interval_bin">
+                <output-dir compare="Text">interval_bin</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="interval_bin_gby_0">
+                <output-dir compare="Text">interval_bin_gby_0</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="interval_bin_gby_1">
+                <output-dir compare="Text">interval_bin_gby_1</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="accessors">
+                <output-dir compare="Text">accessors</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="accessors_interval">
+                <output-dir compare="Text">accessors_interval</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="accessors_interval_null">
+                <output-dir compare="Text">accessors_interval_null</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="adjust_timezone">
+                <output-dir compare="Text">adjust_timezone</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="calendar_duration">
+                <output-dir compare="Text">calendar_duration</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="date_functions">
+                <output-dir compare="Text">date_functions</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="datetime_functions">
+                <output-dir compare="Text">datetime_functions</output-dir>
+            </compilation-unit>
+        </test-case>
+        <!--
+        <test-case FilePath="temporal">
+            <compilation-unit name="insert_from_delimited_ds">
+                <output-dir compare="Text">insert_from_delimited_ds</output-dir>
+            </compilation-unit>
+        </test-case>
+        -->
+        <test-case FilePath="temporal">
+            <compilation-unit name="insert_from_ext_ds">
+                <output-dir compare="Text">insert_from_ext_ds</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="insert_from_ext_ds_2">
+                <output-dir compare="Text">insert_from_ext_ds_2</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="interval_functions">
+                <output-dir compare="Text">interval_functions</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal/interval_joins">
+            <compilation-unit name="interval_overlaps">
+                <output-dir compare="Text">interval_overlaps</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="time_functions">
+                <output-dir compare="Text">time_functions</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="constructor">
+            <compilation-unit name="interval">
+                <output-dir compare="Text">interval</output-dir>
+            </compilation-unit>
+        </test-case>
+        <test-case FilePath="temporal">
+            <compilation-unit name="duration_comps">
+                <output-dir compare="Text">duration_comps</output-dir>
+            </compilation-unit>
+        </test-case>
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.aql
new file mode 100644
index 0000000..f494ebe
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.1.ddl.aql
@@ -0,0 +1,26 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+drop dataverse TinyCollege if exists;
+create dataverse TinyCollege;
+use dataverse TinyCollege;
+
+create type StaffType as open {
+        name: string,
+        office: string,
+        employment: interval
+}
+create dataset Staff(StaffType)
+primary key name;
+
+
+create type StudentType as open {
+        name: string,
+        office: string,
+        attendance: interval
+}
+create dataset Students(StudentType)
+primary key name;
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.aql
new file mode 100644
index 0000000..0b7b8fa
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.2.update.aql
@@ -0,0 +1,22 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+insert into dataset Staff ({"name":"Alex", "office":"A", "employment":interval-from-date(date("2003-01-01"), date("2008-01-01"))});
+insert into dataset Staff ({"name":"Elisabeth", "office":"B", "employment":interval-from-date(date("2002-01-01"), date("2010-01-01"))});
+insert into dataset Staff ({"name":"Frank", "office":"A", "employment":interval-from-date(date("2004-01-01"), date("2009-01-01"))});
+insert into dataset Staff ({"name":"Henry", "office":"C", "employment":interval-from-date(date("2003-01-01"), date("2008-01-01"))});
+insert into dataset Staff ({"name":"Mary", "office":"B", "employment":interval-from-date(date("2006-01-01"), date("2010-01-01"))});
+insert into dataset Staff ({"name":"Vicky", "office":"D", "employment":interval-from-date(date("2001-01-01"), date("2010-01-01"))});
+
+insert into dataset Students ({"name":"Charles", "office":"X", "attendance":interval-from-date(date("2001-01-01"), date("2004-01-01"))});
+insert into dataset Students ({"name":"Frank", "office":"Y", "attendance":interval-from-date(date("2001-01-01"), date("2004-01-01"))});
+insert into dataset Students ({"name":"Karen", "office":"Y", "attendance":interval-from-date(date("2007-01-01"), date("2009-01-01"))});
+insert into dataset Students ({"name":"Mary", "office":"Y", "attendance":interval-from-date(date("2002-01-01"), date("2005-01-01"))});
+insert into dataset Students ({"name":"Olga", "office":"Z", "attendance":interval-from-date(date("2001-01-01"), date("2003-01-01"))});
+insert into dataset Students ({"name":"Steve", "office":"Z", "attendance":interval-from-date(date("2007-01-01"), date("2010-01-01"))});
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.aql
new file mode 100644
index 0000000..c8b2d5e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.3.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where 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/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.aql
new file mode 100644
index 0000000..f8da7f6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.4.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where 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/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
new file mode 100644
index 0000000..22e080b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.5.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [991353600000, 1054425600000, 1149120000000] */ 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/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
new file mode 100644
index 0000000..a4872e5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.6.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $d in dataset Students
+for $f in dataset Staff
+where /*+ interval-merge-join [991353600000, 1054425600000, 1149120000000] */ 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/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
new file mode 100644
index 0000000..7edfc89
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.7.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [991353600000, 1054425600000, 1149120000000] */ 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/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
new file mode 100644
index 0000000..1ec6f4e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/interval_joins/interval_overlaps/interval_overlaps.8.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description      :   Check temporal join functionality for interval
+ * Expected Result  :   Success
+ * Date             :   26th Jun, 2015
+ */
+
+use dataverse TinyCollege;
+
+for $f in dataset Staff
+for $d in dataset Students
+where /*+ interval-merge-join [991353600000, 1054425600000, 1149120000000] */ 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/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
new file mode 100644
index 0000000..a9bd2e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.3.adm
@@ -0,0 +1,6 @@
+[ { "staff": "Alex", "student": "Karen" }
+, { "staff": "Alex", "student": "Steve" }
+, { "staff": "Frank", "student": "Steve" }
+, { "staff": "Henry", "student": "Karen" }
+, { "staff": "Henry", "student": "Steve" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.4.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.4.adm
new file mode 100644
index 0000000..65a07bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.4.adm
@@ -0,0 +1,11 @@
+[ { "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": "Frank", "student": "Mary" }
+, { "staff": "Henry", "student": "Charles" }
+, { "staff": "Henry", "student": "Frank" }
+, { "staff": "Henry", "student": "Mary" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.5.adm
new file mode 100644
index 0000000..a9bd2e4
--- /dev/null
+++ b/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": "Frank", "student": "Steve" }
+, { "staff": "Henry", "student": "Karen" }
+, { "staff": "Henry", "student": "Steve" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.adm
new file mode 100644
index 0000000..65a07bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.6.adm
@@ -0,0 +1,11 @@
+[ { "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": "Frank", "student": "Mary" }
+, { "staff": "Henry", "student": "Charles" }
+, { "staff": "Henry", "student": "Frank" }
+, { "staff": "Henry", "student": "Mary" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm
new file mode 100644
index 0000000..65a07bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.7.adm
@@ -0,0 +1,11 @@
+[ { "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": "Frank", "student": "Mary" }
+, { "staff": "Henry", "student": "Charles" }
+, { "staff": "Henry", "student": "Frank" }
+, { "staff": "Henry", "student": "Mary" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm
new file mode 100644
index 0000000..65a07bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/interval_joins/interval_overlaps/interval_overlaps.8.adm
@@ -0,0 +1,11 @@
+[ { "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": "Frank", "student": "Mary" }
+, { "staff": "Henry", "student": "Charles" }
+, { "staff": "Henry", "student": "Frank" }
+, { "staff": "Henry", "student": "Mary" }
+ ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index d378dca..ba44966 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -19,6 +19,7 @@
 <!DOCTYPE test-suite [
 
 <!ENTITY RecordsQueries SYSTEM "queries/records/RecordsQueries.xml">
+<!ENTITY TemporalQueries SYSTEM "queries/temporal/TemporalQueries.xml">
 
 ]>
 <test-suite
@@ -6094,7 +6095,7 @@
                 <output-dir compare="Text">feeds_04</output-dir>
             </compilation-unit>
         </test-case>
-
+        
         <test-case FilePath="feeds">
           <compilation-unit name="feeds_06">
             <output-dir compare="Text">feeds_06</output-dir>
@@ -6133,7 +6134,7 @@
                 <output-dir compare="Text">feeds_12</output-dir>
             </compilation-unit>
         </test-case>
-
+        
         <test-case FilePath="feeds">
             <compilation-unit name="issue_230_feeds">
                 <output-dir compare="Text">issue_230_feeds</output-dir>
@@ -6149,7 +6150,7 @@
 
     </test-group>
     <test-group name="hdfs">
-    	<test-case FilePath="hdfs">
+        <test-case FilePath="hdfs">
             <compilation-unit name="hdfs_shortcircuit">
                 <output-dir compare="Text">hdfs_shortcircuit</output-dir>
             </compilation-unit>
@@ -6225,143 +6226,7 @@
         </test-case>
     </test-group>
     <test-group name="temporal">
-        <test-case FilePath="temporal">
-            <compilation-unit name="overlap_bins_gby_3">
-                <output-dir compare="Text">overlap_bins_gby_3</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="agg_01">
-                <output-dir compare="Text">agg_01</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="overlap_bins_gby_1">
-                <output-dir compare="Text">overlap_bins_gby_1</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="duration_functions">
-                <output-dir compare="Text">duration_functions</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="overlap_bins_gby_0">
-                <output-dir compare="Text">overlap_bins_gby_0</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="get_overlapping_interval">
-                <output-dir compare="Text">get_overlapping_interval</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="overlap_bins">
-                <output-dir compare="Text">overlap_bins</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="parse_02">
-                <output-dir compare="Text">parse_02</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="parse_01">
-                <output-dir compare="Text">parse_01</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="day_of_week_01">
-                <output-dir compare="Text">day_of_week_01</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="interval_bin">
-                <output-dir compare="Text">interval_bin</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="interval_bin_gby_0">
-                <output-dir compare="Text">interval_bin_gby_0</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="interval_bin_gby_1">
-                <output-dir compare="Text">interval_bin_gby_1</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="accessors">
-                <output-dir compare="Text">accessors</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="accessors_interval">
-                <output-dir compare="Text">accessors_interval</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="accessors_interval_null">
-                <output-dir compare="Text">accessors_interval_null</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="adjust_timezone">
-                <output-dir compare="Text">adjust_timezone</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="calendar_duration">
-                <output-dir compare="Text">calendar_duration</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="date_functions">
-                <output-dir compare="Text">date_functions</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="datetime_functions">
-                <output-dir compare="Text">datetime_functions</output-dir>
-            </compilation-unit>
-        </test-case>
-        <!--
-       <test-case FilePath="temporal">
-           <compilation-unit name="insert_from_delimited_ds">
-              <output-dir compare="Text">insert_from_delimited_ds</output-dir>
-           </compilation-unit>
-      </test-case>
-        -->
-        <test-case FilePath="temporal">
-            <compilation-unit name="insert_from_ext_ds">
-                <output-dir compare="Text">insert_from_ext_ds</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="insert_from_ext_ds_2">
-                <output-dir compare="Text">insert_from_ext_ds_2</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="interval_functions">
-                <output-dir compare="Text">interval_functions</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="time_functions">
-                <output-dir compare="Text">time_functions</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="constructor">
-            <compilation-unit name="interval">
-                <output-dir compare="Text">interval</output-dir>
-            </compilation-unit>
-        </test-case>
-        <test-case FilePath="temporal">
-            <compilation-unit name="duration_comps">
-                <output-dir compare="Text">duration_comps</output-dir>
-            </compilation-unit>
-        </test-case>
+        &TemporalQueries;
     </test-group>
     <test-group name="leftouterjoin">
         <test-case FilePath="leftouterjoin">
diff --git a/asterix-aql/src/main/java/org/apache/asterix/aql/util/RangeMapBuilder.java b/asterix-aql/src/main/java/org/apache/asterix/aql/util/RangeMapBuilder.java
index b18c317..cf20a48 100644
--- a/asterix-aql/src/main/java/org/apache/asterix/aql/util/RangeMapBuilder.java
+++ b/asterix-aql/src/main/java/org/apache/asterix/aql/util/RangeMapBuilder.java
@@ -63,7 +63,9 @@
 
         AQLParser parser = new AQLParser((String) hint);
         List<Statement> hintStatements = parser.parse();
-        if (hintStatements.size() != 1) {
+        if (hintStatements.size() == 0) {
+            throw new ParseException("No range hint was supplied to the RangeMapBuilder.");
+        } else if (hintStatements.size() != 1) {
             throw new ParseException("Only one range statement is allowed for the range hint.");
         }
 
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 0aab549..f1b1b85 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -116,6 +116,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;
@@ -146,7 +147,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";
@@ -442,7 +443,7 @@
     | ("internal" | "temporary" {
             temp = token.image.toLowerCase().equals("temporary");
         }
-      )? 
+      )?
     <DATASET> nameComponents = QualifiedName()
     <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
     ifNotExists = IfNotExists()
@@ -656,7 +657,7 @@
   FunctionSignature appliedFunction = null;
   CreateFeedStatement cfs = null;
   Pair<Identifier,Identifier> sourceNameComponents = null;
-  
+
 }
 {
   (
@@ -668,7 +669,7 @@
       }
      |
      ("primary")? "feed" nameComponents = QualifiedName() ifNotExists = IfNotExists()
-      "using" adapterName = AdapterName() properties = Configuration() (appliedFunction = ApplyFunction())?  
+      "using" adapterName = AdapterName() properties = Configuration() (appliedFunction = ApplyFunction())?
        {
         cfs = new CreatePrimaryFeedStatement(nameComponents,
                                     adapterName, properties, appliedFunction, ifNotExists);
@@ -681,8 +682,8 @@
 
 CreateFeedPolicyStatement FeedPolicySpecification() throws ParseException:
 {
-  String policyName = null;  
-  String basePolicyName = null; 
+  String policyName = null;
+  String basePolicyName = null;
   String sourcePolicyFile = null;
   String definition = null;
   boolean ifNotExists = false;
@@ -692,18 +693,18 @@
 {
   (
     "ingestion" "policy"  policyName = Identifier() ifNotExists = IfNotExists()
-      <FROM> 
-      ("policy" basePolicyName = Identifier() properties = Configuration() ("definition" definition = StringLiteral())?  
+      <FROM>
+      ("policy" basePolicyName = Identifier() properties = Configuration() ("definition" definition = StringLiteral())?
       {
         cfps = new CreateFeedPolicyStatement(policyName,
                                    basePolicyName, properties, definition, ifNotExists);
       }
-     | "path" sourcePolicyFile = Identifier() ("definition" definition = StringLiteral())?  
+     | "path" sourcePolicyFile = Identifier() ("definition" definition = StringLiteral())?
        {
         cfps = new CreateFeedPolicyStatement(policyName, sourcePolicyFile, definition, ifNotExists);
        }
-     ) 
-       
+     )
+
   )
     {
       return cfps;
@@ -2038,7 +2039,13 @@
       }
       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);
+          ijea.setJoinType(hint);
+          ijea.setRangeMap(RangeMapBuilder.parseHint(hint.substring(IntervalJoinExpressionAnnotation.getHintLength(hint))));
+          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);
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java b/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java
new file mode 100644
index 0000000..de1ee1b
--- /dev/null
+++ b/asterix-common/src/main/java/org/apache/asterix/common/annotations/IntervalJoinExpressionAnnotation.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+
+public class IntervalJoinExpressionAnnotation implements IExpressionAnnotation {
+
+    public static final String IOP_HINT_STRING = "interval-iop-join";
+    public static final String MERGE_HINT_STRING = "interval-merge-join";
+    public static final String SPATIAL_HINT_STRING = "interval-spatial-join";
+    public static final IntervalJoinExpressionAnnotation INSTANCE = new IntervalJoinExpressionAnnotation();
+
+    private Object object;
+    private IRangeMap map;
+    private String joinType;
+
+    @Override
+    public Object getObject() {
+        return object;
+    }
+
+    @Override
+    public void setObject(Object object) {
+        this.object = object;
+    }
+
+    @Override
+    public IExpressionAnnotation copy() {
+        IntervalJoinExpressionAnnotation clone = new IntervalJoinExpressionAnnotation();
+        clone.setObject(object);
+        return clone;
+    }
+
+    public void setRangeMap(IRangeMap map) {
+        this.map = map;
+    }
+
+    public IRangeMap getRangeMap() {
+        return map;
+    }
+
+    public void setJoinType(String hint) {
+        if (hint.startsWith(IOP_HINT_STRING)) {
+            joinType = IOP_HINT_STRING;
+        } else if (hint.startsWith(MERGE_HINT_STRING)) {
+            joinType = MERGE_HINT_STRING;
+        } else if (hint.startsWith(SPATIAL_HINT_STRING)) {
+            joinType = SPATIAL_HINT_STRING;
+        }
+    }
+
+    public String getRangeType() {
+        return joinType;
+    }
+
+    public boolean isIopJoin() {
+        if (joinType.equals(IOP_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 static boolean isIntervalJoinHint(String hint) {
+        if (hint.startsWith(IOP_HINT_STRING) || hint.startsWith(MERGE_HINT_STRING)
+                || hint.startsWith(SPATIAL_HINT_STRING)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    public static int getHintLength(String hint) {
+        if (hint.startsWith(IOP_HINT_STRING)) {
+            return IOP_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();
+        }
+        return 0;
+    }
+
+}
diff --git a/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java b/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
index d6da48c..519458e 100644
--- a/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
+++ b/asterix-common/src/main/java/org/apache/asterix/common/annotations/SkipSecondaryIndexSearchExpressionAnnotation.java
@@ -22,7 +22,7 @@
 
 public class SkipSecondaryIndexSearchExpressionAnnotation implements IExpressionAnnotation {
 
-    public static final String SKIP_SECONDARY_INDEX_SEARCH_ANNOTATION_KEY = "skip-index";
+    public static final String HINT_STRING = "skip-index";
     public static final SkipSecondaryIndexSearchExpressionAnnotation INSTANCE = new SkipSecondaryIndexSearchExpressionAnnotation();
 
     private Object object;
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
index f7c4428..aa38010 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
@@ -42,19 +42,20 @@
 
             @Override
             public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-                int c = Double.compare(
+                int c = Long.compare(
                         AInt64SerializerDeserializer.getLong(b1,
                                 s1 + AIntervalSerializerDeserializer.getIntervalStartOffset()),
                         AInt64SerializerDeserializer.getLong(b2,
                                 s2 + AIntervalSerializerDeserializer.getIntervalStartOffset()));
                 if (c == 0) {
-                    c = Double.compare(
+                    c = Long.compare(
                             AInt64SerializerDeserializer.getLong(b1,
                                     s1 + AIntervalSerializerDeserializer.getIntervalEndOffset()),
                             AInt64SerializerDeserializer.getLong(b2,
                                     s2 + AIntervalSerializerDeserializer.getIntervalEndOffset()));
                     if (c == 0) {
-                        c = Byte.compare(b1[s1 + 16], b2[s2 + 16]);
+                        c = Byte.compare(b1[s1 + AIntervalSerializerDeserializer.getIntervalTagOffset()], b2[s2
+                                + AIntervalSerializerDeserializer.getIntervalTagOffset()]);
                     }
                 }
                 return c;
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/AllenRelationsBinaryComparatorFactoryProvider.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/AllenRelationsBinaryComparatorFactoryProvider.java
new file mode 100644
index 0000000..564a793
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/AllenRelationsBinaryComparatorFactoryProvider.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.comparators.allenrelations;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class AllenRelationsBinaryComparatorFactoryProvider implements IBinaryComparatorFactoryProvider, Serializable {
+
+    private static final long serialVersionUID = 1L;
+    public static final AllenRelationsBinaryComparatorFactoryProvider INSTANCE = new AllenRelationsBinaryComparatorFactoryProvider();
+
+    private AllenRelationsBinaryComparatorFactoryProvider() {
+    }
+
+    @Override
+    public IBinaryComparatorFactory getBinaryComparatorFactory(Object type, boolean ascending) {
+        // During a comparison, since proper type promotion among several numeric types are required,
+        // we will use AObjectAscBinaryComparatorFactory, instead of using a specific comparator
+        return OverlapIntervalBinaryComparatorFactory.INSTANCE;
+    }
+
+    public IBinaryComparatorFactory getBinaryComparatorFactory(FunctionIdentifier fid, boolean ascending) {
+        return OverlapIntervalBinaryComparatorFactory.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/OverlapIntervalBinaryComparatorFactory.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/OverlapIntervalBinaryComparatorFactory.java
new file mode 100644
index 0000000..4775c7e
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/allenrelations/OverlapIntervalBinaryComparatorFactory.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.dataflow.data.nontagged.comparators.allenrelations;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class OverlapIntervalBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final OverlapIntervalBinaryComparatorFactory INSTANCE = new OverlapIntervalBinaryComparatorFactory();
+
+    private OverlapIntervalBinaryComparatorFactory() {
+
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                long start0 = AIntervalSerializerDeserializer.getIntervalStart(b1, s1 + 1);
+                long end0 = AIntervalSerializerDeserializer.getIntervalEnd(b1, s1 + 1);
+                long start1 = AIntervalSerializerDeserializer.getIntervalStart(b2, s2 + 1);
+                long end1 = AIntervalSerializerDeserializer.getIntervalEnd(b2, s2 + 1);
+
+                int c = -1;
+                if (start0 < start1 && end0 > start1 && end1 > end0) {
+                    // These intervals overlap
+                    c = 0;
+                } else if (start0 < start1) {
+                    c = 1;
+                }
+                return c;
+            }
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalProjectBinaryComparatorFactory.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalProjectBinaryComparatorFactory.java
new file mode 100644
index 0000000..7a358f2
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalProjectBinaryComparatorFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.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.IBinaryComparatorFactory;
+
+public class RangeIntervalProjectBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final RangeIntervalProjectBinaryComparatorFactory INSTANCE = new RangeIntervalProjectBinaryComparatorFactory();
+
+    private RangeIntervalProjectBinaryComparatorFactory() {
+
+    }
+
+    /*
+     * The comparator uses the range map split value and an interval.
+     *
+     * -1: split point is less than the interval start point.
+     * 0: split point is equal to the interval start point
+     * 1: split point is greater than the interval start point.
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                return Long.compare(AInt64SerializerDeserializer.getLong(b1, s1), AInt64SerializerDeserializer
+                        .getLong(b2, s2 + AIntervalSerializerDeserializer.getIntervalStartOffset()));
+            }
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalReplicateBinaryComparatorFactory.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalReplicateBinaryComparatorFactory.java
new file mode 100644
index 0000000..19fce56
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalReplicateBinaryComparatorFactory.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.IBinaryComparatorFactory;
+
+public class RangeIntervalReplicateBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final RangeIntervalReplicateBinaryComparatorFactory INSTANCE = new RangeIntervalReplicateBinaryComparatorFactory();
+
+    private RangeIntervalReplicateBinaryComparatorFactory() {
+
+    }
+
+    /*
+     * The comparator uses the range map split value and an interval.
+     *
+     * -1: split point is less than the interval start point.
+     * 0: split point is equal to or greater than the interval start point
+     * 1: never happens
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int c = Long.compare(AInt64SerializerDeserializer.getLong(b1, s1), AInt64SerializerDeserializer
+                        .getLong(b2, s2 + AIntervalSerializerDeserializer.getIntervalStartOffset()));
+                if (c > 0) {
+                    c = 0;
+                }
+                return c;
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalSplitBinaryComparatorFactory.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalSplitBinaryComparatorFactory.java
new file mode 100644
index 0000000..8fc2b54
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/comparators/rangeinterval/RangeIntervalSplitBinaryComparatorFactory.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.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.IBinaryComparatorFactory;
+
+public class RangeIntervalSplitBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final RangeIntervalSplitBinaryComparatorFactory INSTANCE = new RangeIntervalSplitBinaryComparatorFactory();
+
+    private RangeIntervalSplitBinaryComparatorFactory() {
+
+    }
+
+    /*
+     * The comparator uses the range map split value and an interval.
+     *
+     * -1: split point is less than the interval start point.
+     * 0: split point is in the interval
+     * 1: split point is greater than the interval end point.
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int c = Double.compare(AInt64SerializerDeserializer.getLong(b1, s1), AInt64SerializerDeserializer
+                        .getLong(b2, s2 + AIntervalSerializerDeserializer.getIntervalStartOffset()));
+                if (c > 0) {
+                    c = Double.compare(AInt64SerializerDeserializer.getLong(b1, s1), AInt64SerializerDeserializer
+                            .getLong(b2, s2 + AIntervalSerializerDeserializer.getIntervalEndOffset()));
+                    if (c < 0) {
+                        c = 0;
+                    }
+                }
+                return c;
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
index 5300b28..d2735b4 100644
--- a/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
@@ -85,13 +85,17 @@
         return 8;
     }
 
+    public static int getIntervalTagOffset() {
+        return 16;
+    }
+
     public static byte getIntervalTimeType(byte[] data, int offset) {
         return data[offset + 8 * 2];
     }
 
     /**
      * create an interval value from two given datetime instance.
-     * 
+     *
      * @param interval
      * @param out
      * @throws HyracksDataException
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java b/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
new file mode 100644
index 0000000..73af3d2
--- /dev/null
+++ b/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/AIntervalPointable.java
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.pointables.nonvisitor;
+
+import org.apache.asterix.om.util.container.IObjectFactory;
+import org.apache.hyracks.api.dataflow.value.ITypeTraits;
+import org.apache.hyracks.data.std.api.AbstractPointable;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.api.IPointableFactory;
+import org.apache.hyracks.data.std.primitive.BytePointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+
+/*
+ * This class interprets the binary data representation of an interval.
+ *
+ * The interval can be time, date, or datetime defined by the tag.
+ *
+ * Interval {
+ *   int startPoint;
+ *   int endPoint;
+ *   byte tag;
+ * }
+ */
+public class AIntervalPointable extends AbstractPointable {
+
+    public static final ITypeTraits TYPE_TRAITS = new ITypeTraits() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public boolean isFixedLength() {
+            return true;
+        }
+
+        @Override
+        public int getFixedLength() {
+            return 17;
+        }
+    };
+
+    public static final IPointableFactory FACTORY = new IPointableFactory() {
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public IPointable createPointable() {
+            return new AIntervalPointable();
+        }
+
+        @Override
+        public ITypeTraits getTypeTraits() {
+            return TYPE_TRAITS;
+        }
+    };
+
+    public static final IObjectFactory<IPointable, String> ALLOCATOR = new IObjectFactory<IPointable, String>() {
+        public IPointable create(String id) {
+            return new AIntervalPointable();
+        }
+    };
+
+    private static final int TAG_SIZE = 1;
+    private static final int START_LENGTH_SIZE = 8;
+    private static final int END_LENGTH_SIZE = 8;
+
+    public long getStart() {
+        return LongPointable.getLong(bytes, getStartOffset());
+    }
+
+    public int getStartOffset() {
+        return start;
+    }
+
+    public int getStartSize() {
+        return START_LENGTH_SIZE;
+    }
+
+    public long getEnd() {
+        return LongPointable.getLong(bytes, getEndOffset());
+    }
+
+    public int getEndOffset() {
+        return getStartOffset() + getStartSize();
+    }
+
+    public int getEndSize() {
+        return END_LENGTH_SIZE;
+    }
+
+    public byte getTag() {
+        return BytePointable.getByte(bytes, getTagOffset());
+    }
+
+    public int getTagOffset() {
+        return getEndOffset() + getEndSize();
+    }
+
+    public int getTagSize() {
+        return TAG_SIZE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
index f6e3183..7b2e8f0 100644
--- a/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
+++ b/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AIntervalTypeComputer.java
@@ -27,7 +27,7 @@
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 
 public class AIntervalTypeComputer implements IResultTypeComputer {
-
+    private static final long serialVersionUID = 1L;
     public static final AIntervalTypeComputer INSTANCE = new AIntervalTypeComputer();
 
     private AIntervalTypeComputer() {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
index 7f731d4..3bbbf27 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
@@ -21,7 +21,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.asterix.common.functions.FunctionConstants;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import org.apache.asterix.om.base.AInterval;
@@ -151,8 +150,8 @@
                             }
 
                             if (intervalEnd < intervalStart) {
-                                throw new AlgebricksException(FID.getName()
-                                        + ": interval end must not be less than the interval start.");
+                                throw new AlgebricksException(
+                                        FID.getName() + ": interval end must not be less than the interval start.");
                             }
 
                             aInterval.setValue(intervalStart, intervalEnd, ATypeTag.DATETIME.serialize());
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
index 5678e10..032909f 100644
--- a/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/temporal/GetOverlappingIntervalDescriptor.java
@@ -114,7 +114,7 @@
                                         1);
                                 long end1 = AIntervalSerializerDeserializer.getIntervalEnd(argOut1.getByteArray(), 1);
 
-                                if (IntervalLogic.overlap(start0, end0, start1, end1)
+                                if (IntervalLogic.overlaps(start0, end0, start1, end1)
                                         || IntervalLogic.overlappedBy(start0, end0, start1, end1)
                                         || IntervalLogic.covers(start0, end0, start1, end1)
                                         || IntervalLogic.coveredBy(start0, end0, start1, end1)) {
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..e652217
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/AbstractIntervalMergeJoinChecker.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinChecker;
+
+public abstract class AbstractIntervalMergeJoinChecker implements IMergeJoinChecker {
+
+    private static final long serialVersionUID = 1L;
+    protected final int idLeft;
+    protected final int idRight;
+
+    public AbstractIntervalMergeJoinChecker(int idLeft, int idRight) {
+        this.idLeft = idLeft;
+        this.idRight = idRight;
+    }
+
+    protected long getIntervalStart(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+            throws HyracksDataException {
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId) + 1;
+        return AIntervalSerializerDeserializer.getIntervalStart(accessor.getBuffer().array(), start);
+    }
+
+    protected long getIntervalEnd(IFrameTupleAccessor accessor, int tupleId, int fieldId) throws HyracksDataException {
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId) + 1;
+        return AIntervalSerializerDeserializer.getIntervalEnd(accessor.getBuffer().array(), start);
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..00d8805
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinChecker.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.runtime.evaluators.functions.temporal.IntervalLogic;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class OverlappedByIntervalMergeJoinChecker extends AbstractIntervalMergeJoinChecker {
+    private static final long serialVersionUID = 1L;
+
+    public OverlappedByIntervalMergeJoinChecker(int[] keysLeft, int[] keysRight) {
+        super(keysLeft[0], keysRight[0]);
+    }
+
+    public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        return (start1 < end0);
+    }
+
+    public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long start0 = getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+        long end1 = getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+        return (end1 < start0);
+    }
+
+    public boolean checkToLoadNextRightTuple(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        return (start1 < end0);
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long start0 = getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        long end1 = getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+
+        return IntervalLogic.overlappedBy(start0, end0, start1, end1);
+    }
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..1bd9bca
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlappedByIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinChecker;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+
+public class OverlappedByIntervalMergeJoinCheckerFactory implements IMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IMergeJoinChecker createMergeJoinChecker(IBinaryComparatorFactory[] comparatorFactories, int[] keys0,
+            int[] keys1) {
+        return new OverlappedByIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.java
new file mode 100644
index 0000000..5913ee9
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinChecker.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.runtime.evaluators.functions.temporal.IntervalLogic;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+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]);
+    }
+
+    public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        return (start1 < end0);
+    }
+
+    public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long start0 = getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+        long end1 = getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+        return (end1 < start0);
+    }
+
+    public boolean checkToLoadNextRightTuple(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        return (start1 < end0);
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+            IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+        long start0 = getIntervalStart(accessorLeft, leftTupleIndex, idLeft);
+        long end0 = getIntervalEnd(accessorLeft, leftTupleIndex, idLeft);
+
+        long start1 = getIntervalStart(accessorRight, rightTupleIndex, idRight);
+        long end1 = getIntervalEnd(accessorRight, rightTupleIndex, idRight);
+
+        return IntervalLogic.overlaps(start0, end0, start1, end1);
+    }
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..69f4b71
--- /dev/null
+++ b/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/OverlapsIntervalMergeJoinCheckerFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinChecker;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+
+public class OverlapsIntervalMergeJoinCheckerFactory implements IMergeJoinCheckerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public IMergeJoinChecker createMergeJoinChecker(IBinaryComparatorFactory[] comparatorFactories, int[] keys0,
+            int[] keys1) {
+        return new OverlapsIntervalMergeJoinChecker(keys0, keys1);
+    }
+
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index 3a6ff21..9e59bd8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -8,7 +8,7 @@
  ! "License"); you may not use this file except in compliance
  ! with the License.  You may obtain a copy of the License at
  !
- !   http://www.apache.org/licenses/LICENSE-2.0
+ !     http://www.apache.org/licenses/LICENSE-2.0
  !
  ! Unless required by applicable law or agreed to in writing,
  ! software distributed under the License is distributed on an
@@ -56,7 +56,7 @@
         <invalid.tests>**/DmlTest.java</invalid.tests>
         <global.test.includes>**/*TestSuite.java,**/*Test.java,${execution.tests}</global.test.includes>
         <global.test.excludes>${optimizer.tests},${metadata.tests},${invalid.tests},${repeated.tests}</global.test.excludes>
-    <!-- Versions under dependencymanagement or used in many projects via properties -->
+    <!-- Versions under dependency management or used in many projects via properties -->
         <algebricks.version>0.2.17-SNAPSHOT</algebricks.version>
         <hyracks.version>0.2.17-SNAPSHOT</hyracks.version>
         <hadoop.version>2.2.0</hadoop.version>
@@ -146,31 +146,6 @@
                         <include>**/*.java</include>
                     </includes>
                 </configuration>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-assembly-plugin</artifactId>
-                 <!-- We override the configuration plugin to override the descriptor to use for building
-                  the source release zip. Specifically, we would like to control the inclusions/exclusions.
-                  For example, we exclude the KEYS file from the zip -->
-                <executions>
-                    <execution>
-                    <!-- Use this id to match the id mentioned in the assembly plugin configuration in
-                     the apache parent POM under the apache-release profile -->
-                        <id>source-release-assembly</id>
-                        <phase>package</phase>
-                        <goals>
-                            <goal>single</goal>
-                        </goals>
-                        <!-- combine.self should be override to replace the configuration in the parent POM -->
-                        <configuration combine.self="override">
-                            <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
-                            <descriptors>
-                                <descriptor>src/main/assembly/source.xml</descriptor>
-                            </descriptors>
-                        </configuration>
-                  </execution>
-              </executions>
             </plugin>
             <plugin>
                 <groupId>org.apache.maven.plugins</groupId>
@@ -433,6 +408,15 @@
             <id>algebricks-snapshots</id>
             <url>http://obelix.ics.uci.edu/nexus/content/repositories/algebricks-snapshots/</url>
         </repository>
+        <repository>
+            <snapshots>
+                <enabled>true</enabled>
+                <updatePolicy>always</updatePolicy>
+                <checksumPolicy>fail</checksumPolicy>
+            </snapshots>
+            <id>apache-staging</id>
+            <url>https://repository.apache.org/content/repositories/orgapacheasterix-1005/</url>
+        </repository>
     </repositories>
     <dependencyManagement>
         <dependencies>

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/457
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Id4f9be43eb0c500f04f99438c34e69503591641d
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <prestonc@apache.org>

Mime
View raw message