[ASTERIXDB-2839] Added Optimized Spatial Joins

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

- This is an implementation of PBSM algorithm to improve performance of SQL++ queries with spatial realated condition: spatial_intersect, st_intersects, st_contains, st_distance, etc.
- Add spatial partitioning hint. The optimization process would be only triggered if the hint was provided.
- Update Optimizer tests for new spatial joins.
- Update spatial join runtime tests.

Change-Id: I237eaf73c531709260adcefde26ef4257c54f513
Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/13323
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java
new file mode 100644
index 0000000..c0d9f4b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/SpatialJoinPOperator.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.asterix.runtime.operators.joins.spatial.PlaneSweepJoinOperatorDescriptor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+
+/**
+ * The right input is broadcast and the left input can be partitioned in any way.
+ */
+public class SpatialJoinPOperator extends AbstractJoinPOperator {
+
+    private final List<LogicalVariable> keysLeftBranch;
+    private final List<LogicalVariable> keysRightBranch;
+
+    protected final ISpatialJoinUtilFactory mjcf;
+    private final int memSizeInFrames;
+
+    public SpatialJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType,
+            List<LogicalVariable> keysLeftBranch, List<LogicalVariable> keysRightBranch, int memSizeInFrames,
+            ISpatialJoinUtilFactory mjcf) {
+        super(kind, partitioningType);
+        this.keysLeftBranch = keysLeftBranch;
+        this.keysRightBranch = keysRightBranch;
+        this.mjcf = mjcf;
+        this.memSizeInFrames = memSizeInFrames;
+    }
+
+    public List<LogicalVariable> getKeysLeftBranch() {
+        return keysLeftBranch;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.SPATIAL_JOIN;
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return "SPATIAL_JOIN" + " " + keysLeftBranch + " " + keysRightBranch;
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context)
+            throws AlgebricksException {
+        IPartitioningProperty pp;
+        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.LOCAL) {
+            AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+            IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+            AbstractLogicalOperator op1 = (AbstractLogicalOperator) op.getInputs().get(1).getValue();
+            IPhysicalPropertiesVector pv1 = op1.getPhysicalOperator().getDeliveredProperties();
+
+            if (pv0 == null || pv1 == null) {
+                pp = null;
+            } else {
+                pp = pv0.getPartitioningProperty();
+            }
+        } else {
+            pp = IPartitioningProperty.UNPARTITIONED;
+        }
+        this.deliveredProperties = new StructuralPropertiesVector(pp, deliveredLocalProperties(iop, context));
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        List<LogicalVariable> keysLeftBranchTileId = new ArrayList<>();
+        keysLeftBranchTileId.add(keysLeftBranch.get(0));
+        List<LogicalVariable> keysRightBranchTileId = new ArrayList<>();
+        keysRightBranchTileId.add(keysRightBranch.get(0));
+        IPartitioningProperty pp1 = new UnorderedPartitionedProperty(new ListSet<>(keysLeftBranchTileId),
+                context.getComputationNodeDomain());
+        IPartitioningProperty pp2 = new UnorderedPartitionedProperty(new ListSet<>(keysRightBranchTileId),
+                context.getComputationNodeDomain());
+
+        List<ILocalStructuralProperty> localProperties1 = new ArrayList<>();
+        List<OrderColumn> orderColumns1 = new ArrayList<OrderColumn>();
+        orderColumns1.add(new OrderColumn(keysLeftBranch.get(0), OrderOperator.IOrder.OrderKind.ASC));
+        orderColumns1.add(new OrderColumn(keysLeftBranch.get(1), OrderOperator.IOrder.OrderKind.ASC));
+        localProperties1.add(new LocalOrderProperty(orderColumns1));
+
+        List<ILocalStructuralProperty> localProperties2 = new ArrayList<>();
+        List<OrderColumn> orderColumns2 = new ArrayList<OrderColumn>();
+        orderColumns2.add(new OrderColumn(keysRightBranch.get(0), OrderOperator.IOrder.OrderKind.ASC));
+        orderColumns2.add(new OrderColumn(keysRightBranch.get(1), OrderOperator.IOrder.OrderKind.ASC));
+        localProperties2.add(new LocalOrderProperty(orderColumns2));
+
+        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+        pv[0] = new StructuralPropertiesVector(pp1, localProperties1);
+        pv[1] = new StructuralPropertiesVector(pp2, localProperties2);
+
+        return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        int[] keysBuild = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+        int[] keysProbe = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+
+        IOperatorDescriptorRegistry spec = builder.getJobSpec();
+        RecordDescriptor recordDescriptor =
+                JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+
+        IOperatorDescriptor opDesc = new PlaneSweepJoinOperatorDescriptor(spec, memSizeInFrames, keysBuild, keysProbe,
+                recordDescriptor, mjcf);
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+        ILogicalOperator src1 = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src1, 0, op, 0);
+        ILogicalOperator src2 = op.getInputs().get(1).getValue();
+        builder.contributeGraphEdge(src2, 0, op, 1);
+    }
+
+    protected List<ILocalStructuralProperty> deliveredLocalProperties(ILogicalOperator op,
+            IOptimizationContext context) {
+        AbstractLogicalOperator op0 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        IPhysicalPropertiesVector pv0 = op0.getPhysicalOperator().getDeliveredProperties();
+        List<ILocalStructuralProperty> lp0 = pv0.getLocalProperties();
+        if (lp0 != null) {
+            // maintains the local properties on the probe side
+            return new LinkedList<>(lp0);
+        }
+        return new LinkedList<>();
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index d721c31..1641665 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -45,6 +45,8 @@
 import org.apache.asterix.optimizer.rules.ExtractOrderExpressionsRule;
 import org.apache.asterix.optimizer.rules.ExtractWindowExpressionsRule;
 import org.apache.asterix.optimizer.rules.FeedScanCollectionToUnnest;
+import org.apache.asterix.optimizer.rules.FilterRefineSpatialJoinRuleForSTDistanceFunction;
+import org.apache.asterix.optimizer.rules.FilterRefineSpatialJoinRuleForSTFunctions;
 import org.apache.asterix.optimizer.rules.FindDataSourcesRule;
 import org.apache.asterix.optimizer.rules.FixReplicateOperatorOutputsRule;
 import org.apache.asterix.optimizer.rules.FullTextContainsParameterCheckAndSetRule;
@@ -66,6 +68,7 @@
 import org.apache.asterix.optimizer.rules.LoadRecordFieldsRule;
 import org.apache.asterix.optimizer.rules.MetaFunctionToMetaVariableRule;
 import org.apache.asterix.optimizer.rules.NestGroupByRule;
+import org.apache.asterix.optimizer.rules.PullSelectOutOfSpatialJoin;
 import org.apache.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import org.apache.asterix.optimizer.rules.PushAggregateIntoNestedSubplanRule;
 import org.apache.asterix.optimizer.rules.PushFieldAccessRule;
@@ -279,6 +282,8 @@
         fieldLoads.add(new NestedSubplanToJoinRule());
         fieldLoads.add(new InlineSubplanInputForNestedTupleSourceRule());
         fieldLoads.add(new RemoveLeftOuterUnnestForLeftOuterJoinRule());
+        fieldLoads.add(new FilterRefineSpatialJoinRuleForSTFunctions());
+        fieldLoads.add(new FilterRefineSpatialJoinRuleForSTDistanceFunction());
         return fieldLoads;
     }
 
@@ -374,6 +379,8 @@
         physicalRewritesAllLevels.add(new ConsolidateAssignsRule(true));
         // After adding projects, we may need need to set physical operators again.
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
+        // Optimized spatial join's query plan produces more join conditions, so we need to pull out these conditions
+        physicalRewritesAllLevels.add(new PullSelectOutOfSpatialJoin());
         return physicalRewritesAllLevels;
     }
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
index 0124d25..9cbed97 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/AsterixExtractFunctionsFromJoinConditionRule.java
@@ -31,7 +31,7 @@
 
     @Override
     protected boolean isComparisonFunction(FunctionIdentifier fi) {
-        return BuiltinFunctions.isSimilarityFunction(fi);
+        return BuiltinFunctions.isSimilarityFunction(fi) || BuiltinFunctions.isSpatialFilterFunction(fi);
     }
 
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.java
new file mode 100644
index 0000000..254e254
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTDistanceFunction.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;
+
+import java.util.List;
+
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join condition is st_distance(), this rule applies the spatial join into the query
+ * by adding the spatial-intersect function and sends the extended mbr of the geometries to it.
+ *
+ * For example:<br/>
+ *
+ * join (lt(st-distance($$54, $$55), 1)) -- |UNPARTITIONED|
+ *   assign [$$54] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$48, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *   assign [$$55] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$49, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *
+ * Becomes,
+ *
+ * join (and(spatial-intersect(st-mbr-enlarge($$54, 1), st-mbr($$55)), lt(st-distance($$54, $$55), 1))) -- |UNPARTITIONED|
+ *   assign [$$54] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$48, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *   assign [$$55] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$49, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *
+ * st-mbr() computes the mbr of a Geometry, and st-mbr-enlarge() computes the mbr of a Geometry and extending
+ * by the second parameter.
+ *
+ * The /*+ spatial-partitioning(x1, y1, x2, y2, row, col) &#42;/ annotation allows users to define the MBR and
+ * grid size (row,col) and these are used for partitioning. If the query does not have an annotation, the MBR is
+ * computed dynamically and grid size set to 100 100.
+ */
+public class FilterRefineSpatialJoinRuleForSTDistanceFunction implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        Mutable<ILogicalExpression> joinConditionRef = joinOp.getCondition();
+        ILogicalExpression joinCondition = joinConditionRef.getValue();
+
+        if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+        if (!(funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.LT)
+                || funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.LE)
+                || funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.EQ))) {
+            return false;
+        }
+
+        List<Mutable<ILogicalExpression>> inputExprs = funcExpr.getArguments();
+
+        ILogicalExpression leftOperatingExpr = inputExprs.get(0).getValue();
+        ILogicalExpression rightOperatingExpr = inputExprs.get(1).getValue();
+
+        if (leftOperatingExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL
+                || rightOperatingExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+
+        AbstractFunctionCallExpression distanceFuncCallExpr = (AbstractFunctionCallExpression) leftOperatingExpr;
+        ConstantExpression distanceValExpr = (ConstantExpression) rightOperatingExpr;
+
+        if (!distanceFuncCallExpr.getFunctionIdentifier().equals(BuiltinFunctions.ST_DISTANCE)) {
+            return false;
+        }
+
+        // Left and right arguments of the st_distance function should be either variable or function call.
+        List<Mutable<ILogicalExpression>> distanceFuncCallArgs = distanceFuncCallExpr.getArguments();
+        Mutable<ILogicalExpression> distanceFuncCallLeftArg = distanceFuncCallArgs.get(0);
+        Mutable<ILogicalExpression> distanceFuncCallRightArg = distanceFuncCallArgs.get(1);
+        if (distanceFuncCallLeftArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT
+                || distanceFuncCallRightArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+
+        // Enlarge the MBR of the left argument of the refine function (st_distance)
+        IAlgebricksConstantValue distanceVal = distanceValExpr.getValue();
+        ScalarFunctionCallExpression enlargedLeft = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR_ENLARGE), distanceFuncCallLeftArg,
+                new MutableObject<>(new ConstantExpression(distanceVal)));
+        enlargedLeft.setSourceLocation(distanceFuncCallLeftArg.getValue().getSourceLocation());
+        // Compute the MBR of the right argument of the refine function (st_distance)
+        ScalarFunctionCallExpression rightMBR = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), distanceFuncCallRightArg);
+        rightMBR.setSourceLocation(distanceFuncCallRightArg.getValue().getSourceLocation());
+
+        // Create filter function (spatial_intersect)
+        ScalarFunctionCallExpression spatialIntersect = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_INTERSECT),
+                new MutableObject<>(enlargedLeft.cloneExpression()), new MutableObject<>(rightMBR.cloneExpression()));
+        spatialIntersect.setSourceLocation(op.getSourceLocation());
+
+        // Attach the annotation to the spatial_intersect function if it is available
+        if (distanceFuncCallExpr.getAnnotation(SpatialJoinAnnotation.class) != null) {
+            spatialIntersect.putAnnotation(distanceFuncCallExpr.getAnnotation(SpatialJoinAnnotation.class));
+        }
+
+        // Update join condition with filter and refine function
+        ScalarFunctionCallExpression updatedJoinCondition =
+                new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+                        new MutableObject<>(spatialIntersect), new MutableObject<>(funcExpr));
+        updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+
+        joinConditionRef.setValue(updatedJoinCondition);
+
+        return true;
+    }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java
new file mode 100644
index 0000000..fecd91c
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FilterRefineSpatialJoinRuleForSTFunctions.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.List;
+
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join condition is one of the below Geometry functions,
+ * this rule applies the spatial join by adding the spatial-intersect condition into the join predicate.
+ *
+ * <ul>
+ *     <li>st_intersects</li>
+ *     <li>st_overlaps</li>
+ *     <li>st_touches</li>
+ *     <li>st_contains</li>
+ *     <li>st_crosses</li>
+ *     <li>st_within</li>
+ * </ul>
+ *
+ *
+ * For example:<br/>
+ *
+ * join (st-intersects($$50, $$51)) -- |UNPARTITIONED|
+ *   assign [$$50] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$47, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *   assign [$$51] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$48, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *
+ * Becomes,
+ *
+ * join (and(spatial-intersect(st-mbr($$50), st-mbr($$51)), st-intersects($$50, $$51))) -- |UNPARTITIONED|
+ *   assign [$$50] <- [$$ps.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$47, $$ps] <- test.ParkSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *   assign [$$51] <- [$$ls.getField(1)] -- |UNPARTITIONED|
+ *     data-scan []<-[$$48, $$ls] <- test.LakeSetG -- |UNPARTITIONED|
+ *       empty-tuple-source -- |UNPARTITIONED|
+ *
+ *  st-mbr($x, $y) computes the mbr of the geometry and returns rectangles to pass it spatial_intersect($x, $y)
+ *
+ *  The /*+ spatial-partitioning(x1, y1, x2, y2, row, col) &#42;/ annotation allows users to define the MBR and
+ *  grid size (row,col) which are used for partitioning. If the query does not have an annotation, the MBR is computed
+ *  dynamically and grid size set to 100 100.
+ */
+public class FilterRefineSpatialJoinRuleForSTFunctions implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        Mutable<ILogicalExpression> joinConditionRef = joinOp.getCondition();
+        ILogicalExpression joinCondition = joinConditionRef.getValue();
+
+        if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+
+        AbstractFunctionCallExpression stFuncExpr = (AbstractFunctionCallExpression) joinCondition;
+        if (!BuiltinFunctions.isSTFilterRefineFunction(stFuncExpr.getFunctionIdentifier())) {
+            return false;
+        }
+
+        // Left and right arguments of the refine function should be either variable or function call.
+        List<Mutable<ILogicalExpression>> stFuncArgs = stFuncExpr.getArguments();
+        Mutable<ILogicalExpression> stFuncLeftArg = stFuncArgs.get(0);
+        Mutable<ILogicalExpression> stFuncRightArg = stFuncArgs.get(1);
+        if (stFuncLeftArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT
+                || stFuncRightArg.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            return false;
+        }
+
+        // Compute MBRs of the left and right arguments of the refine function
+        ScalarFunctionCallExpression left = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), stFuncLeftArg);
+        left.setSourceLocation(stFuncLeftArg.getValue().getSourceLocation());
+        ScalarFunctionCallExpression right = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.ST_MBR), stFuncRightArg);
+        right.setSourceLocation(stFuncRightArg.getValue().getSourceLocation());
+
+        // Create filter function (spatial_intersect)
+        ScalarFunctionCallExpression spatialIntersect = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_INTERSECT),
+                new MutableObject<>(left.cloneExpression()), new MutableObject<>(right.cloneExpression()));
+        spatialIntersect.setSourceLocation(op.getSourceLocation());
+
+        // Attach the annotation to the spatial_intersect function if it is available
+        if (stFuncExpr.getAnnotation(SpatialJoinAnnotation.class) != null) {
+            spatialIntersect.putAnnotation(stFuncExpr.getAnnotation(SpatialJoinAnnotation.class));
+        }
+
+        // Update join condition with filter and refine function
+        ScalarFunctionCallExpression updatedJoinCondition =
+                new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND),
+                        new MutableObject<>(spatialIntersect), new MutableObject<>(stFuncExpr));
+        updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+        joinConditionRef.setValue(updatedJoinCondition);
+
+        return true;
+    }
+
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
new file mode 100644
index 0000000..9653793
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PullSelectOutOfSpatialJoin.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * If the join predicate contains multiple conditions, in which one of them is spatial_intersect($x, $y), all other
+ * condition will be pull out to a SELECT operator after the SPATIAL_JOIN operator.
+ *
+ * For example:<br/>
+ * join (and(spatial-intersect($$52, $$53), lt(st-distance($$56, $$57), 1.0))
+ * -- SPATIAL_JOIN [$$62, $$52] [$$63, $$53]  |PARTITIONED|
+ *
+ * Becomes,
+ *
+ * select (lt(st-distance($$56, $$57), 1.0))
+ * -- STREAM_SELECT  |PARTITIONED|
+ *     exchange
+ *     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+ *         join (spatial-intersect($$52, $$53))
+ *         -- SPATIAL_JOIN [$$62, $$52] [$$63, $$53]  |PARTITIONED|
+ */
+
+public class PullSelectOutOfSpatialJoin implements IAlgebraicRewriteRule {
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return false;
+        }
+        AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
+
+        if (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SPATIAL_JOIN) {
+            return false;
+        }
+
+        ILogicalExpression expr = join.getCondition().getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier fi = fexp.getFunctionIdentifier();
+        if (!fi.equals(AlgebricksBuiltinFunctions.AND)) {
+            return false;
+        }
+        List<Mutable<ILogicalExpression>> spatialVarVarComps = new ArrayList<Mutable<ILogicalExpression>>();
+        List<Mutable<ILogicalExpression>> otherPredicates = new ArrayList<Mutable<ILogicalExpression>>();
+        for (Mutable<ILogicalExpression> arg : fexp.getArguments()) {
+            if (isSpatialVarVar(arg.getValue(), join, context)) {
+                spatialVarVarComps.add(arg);
+            } else {
+                otherPredicates.add(arg);
+            }
+        }
+        if (spatialVarVarComps.isEmpty() || otherPredicates.isEmpty()) {
+            return false;
+        }
+        // pull up
+        ILogicalExpression pulledCond = makeCondition(otherPredicates, context, op);
+        SelectOperator select = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond), false, null);
+        select.setSourceLocation(op.getSourceLocation());
+        ILogicalExpression newJoinCond = makeCondition(spatialVarVarComps, context, op);
+        join.getCondition().setValue(newJoinCond);
+        select.getInputs().add(new MutableObject<ILogicalOperator>(join));
+        context.computeAndSetTypeEnvironmentForOperator(select);
+        select.recomputeSchema();
+        opRef.setValue(select);
+
+        return true;
+    }
+
+    private ILogicalExpression makeCondition(List<Mutable<ILogicalExpression>> predList, IOptimizationContext context,
+            AbstractLogicalOperator op) {
+        if (predList.size() > 1) {
+            IFunctionInfo finfo = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
+            ScalarFunctionCallExpression conditionExpr = new ScalarFunctionCallExpression(finfo, predList);
+            conditionExpr.setSourceLocation(op.getSourceLocation());
+            return conditionExpr;
+        } else {
+            return predList.get(0).getValue();
+        }
+    }
+
+    private boolean isSpatialVarVar(ILogicalExpression expr, AbstractBinaryJoinOperator join,
+            IOptimizationContext context) throws AlgebricksException {
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+        if (!f.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+            return false;
+        }
+
+        // We only apply this rule if the arguments of spatial_intersect are ARectangle
+        IVariableTypeEnvironment typeEnvironment = join.computeInputTypeEnvironment(context);
+        IAType leftType = (IAType) context.getExpressionTypeComputer().getType(f.getArguments().get(0).getValue(),
+                context.getMetadataProvider(), typeEnvironment);
+        IAType rightType = (IAType) context.getExpressionTypeComputer().getType(f.getArguments().get(1).getValue(),
+                context.getMetadataProvider(), typeEnvironment);
+        if ((leftType != BuiltinType.ARECTANGLE) || (rightType != BuiltinType.ARECTANGLE)) {
+            return false;
+        }
+
+        ILogicalExpression e1 = f.getArguments().get(0).getValue();
+        if (e1.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return false;
+        } else {
+            ILogicalExpression e2 = f.getArguments().get(1).getValue();
+            return e2.getExpressionTag() == LogicalExpressionTag.VARIABLE;
+        }
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
index 672cb25..5781d6c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/AsterixJoinUtils.java
@@ -18,29 +18,14 @@
  */
 package org.apache.asterix.optimizer.rules.util;
 
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.asterix.common.annotations.RangeAnnotation;
-import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
-import org.apache.hyracks.api.exceptions.ErrorCode;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
-import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class AsterixJoinUtils {
 
-    private static final int LEFT = 0;
-    private static final int RIGHT = 1;
-
     private AsterixJoinUtils() {
     }
 
@@ -49,37 +34,14 @@
         if (!topLevelOp) {
             return;
         }
-        ILogicalExpression conditionLE = op.getCondition().getValue();
-        if (conditionLE.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+        ILogicalExpression joinCondition = op.getCondition().getValue();
+        if (joinCondition.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return;
         }
-        List<LogicalVariable> sideLeft = new ArrayList<>(1);
-        List<LogicalVariable> sideRight = new ArrayList<>(1);
-        List<LogicalVariable> varsLeft = op.getInputs().get(LEFT).getValue().getSchema();
-        List<LogicalVariable> varsRight = op.getInputs().get(RIGHT).getValue().getSchema();
-        AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) conditionLE;
-        FunctionIdentifier fi =
-                IntervalJoinUtils.isIntervalJoinCondition(fexp, varsLeft, varsRight, sideLeft, sideRight, LEFT, RIGHT);
-        if (fi == null) {
-            return;
+
+        boolean intervalJoinChosen = IntervalJoinUtils.tryIntervalJoinAssignment(op, context, joinCondition, 0, 1);
+        if (!intervalJoinChosen) {
+            SpatialJoinUtils.trySpatialJoinAssignment(op, context, joinCondition, 0, 1);
         }
-        RangeAnnotation rangeAnnotation = IntervalJoinUtils.findRangeAnnotation(fexp);
-        if (rangeAnnotation == null) {
-            return;
-        }
-        //Check RangeMap type
-        RangeMap rangeMap = rangeAnnotation.getRangeMap();
-        if (rangeMap.getTag(0, 0) != ATypeTag.DATETIME.serialize() && rangeMap.getTag(0, 0) != ATypeTag.DATE.serialize()
-                && rangeMap.getTag(0, 0) != ATypeTag.TIME.serialize()) {
-            IWarningCollector warningCollector = context.getWarningCollector();
-            if (warningCollector.shouldWarn()) {
-                warningCollector.warn(Warning.of(op.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
-                        "Date, DateTime, and Time are only range hints types supported for interval joins"));
-            }
-            return;
-        }
-        IntervalPartitions intervalPartitions =
-                IntervalJoinUtils.createIntervalPartitions(op, fi, sideLeft, sideRight, rangeMap, context, LEFT, RIGHT);
-        IntervalJoinUtils.setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, context, intervalPartitions);
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
index e10f9fb..340c841 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.runtime.operators.joins.interval.utils.AfterIntervalJoinUtilFactory;
 import org.apache.asterix.runtime.operators.joins.interval.utils.BeforeIntervalJoinUtilFactory;
 import org.apache.asterix.runtime.operators.joins.interval.utils.CoveredByIntervalJoinUtilFactory;
@@ -60,6 +61,8 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty.PartitioningType;
 import org.apache.hyracks.algebricks.core.algebra.properties.IntervalColumn;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class IntervalJoinUtils {
@@ -76,6 +79,44 @@
         INTERVAL_JOIN_CONDITIONS.put(BuiltinFunctions.INTERVAL_OVERLAPS, BuiltinFunctions.INTERVAL_OVERLAPPED_BY);
     }
 
+    protected static boolean tryIntervalJoinAssignment(AbstractBinaryJoinOperator op, IOptimizationContext context,
+            ILogicalExpression joinCondition, int left, int right) throws AlgebricksException {
+        List<LogicalVariable> sideLeft = new ArrayList<>(1);
+        List<LogicalVariable> sideRight = new ArrayList<>(1);
+        List<LogicalVariable> varsLeft = op.getInputs().get(left).getValue().getSchema();
+        List<LogicalVariable> varsRight = op.getInputs().get(right).getValue().getSchema();
+
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+        FunctionIdentifier fi = IntervalJoinUtils.isIntervalJoinCondition(funcExpr, varsLeft, varsRight, sideLeft,
+                sideRight, left, right);
+        if (fi == null) {
+            return false;
+        }
+
+        // Existing workflow for interval merge join
+        RangeAnnotation rangeAnnotation = IntervalJoinUtils.findRangeAnnotation(funcExpr);
+        if (rangeAnnotation == null) {
+            return false;
+        }
+
+        //Check RangeMap type
+        RangeMap rangeMap = rangeAnnotation.getRangeMap();
+        if (rangeMap.getTag(0, 0) != ATypeTag.DATETIME.serialize() && rangeMap.getTag(0, 0) != ATypeTag.DATE.serialize()
+                && rangeMap.getTag(0, 0) != ATypeTag.TIME.serialize()) {
+            IWarningCollector warningCollector = context.getWarningCollector();
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(op.getSourceLocation(),
+                        org.apache.hyracks.api.exceptions.ErrorCode.INAPPLICABLE_HINT,
+                        "Date, DateTime, and Time are only range hints types supported for interval joins"));
+            }
+            return false;
+        }
+        IntervalPartitions intervalPartitions =
+                IntervalJoinUtils.createIntervalPartitions(op, fi, sideLeft, sideRight, rangeMap, context, left, right);
+        IntervalJoinUtils.setSortMergeIntervalJoinOp(op, fi, sideLeft, sideRight, context, intervalPartitions);
+        return true;
+    }
+
     protected static RangeAnnotation findRangeAnnotation(AbstractFunctionCallExpression fexp) {
         return fexp.getAnnotation(RangeAnnotation.class);
     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java
new file mode 100644
index 0000000..16135e5
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SpatialJoinUtils.java
@@ -0,0 +1,655 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.asterix.algebra.operators.physical.SpatialJoinPOperator;
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
+import org.apache.asterix.om.base.ABoolean;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.IntersectSpatialJoinUtilFactory;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.common.utils.Triple;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.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.AbstractLogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.NestedLoopJoinPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.UnnestPOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class SpatialJoinUtils {
+
+    private static final int DEFAULT_ROWS = 100;
+    private static final int DEFAULT_COLUMNS = 100;
+
+    protected static boolean trySpatialJoinAssignment(AbstractBinaryJoinOperator op, IOptimizationContext context,
+            ILogicalExpression joinCondition, int left, int right) throws AlgebricksException {
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) joinCondition;
+        // Check if the join condition contains spatial join
+        AbstractFunctionCallExpression spatialJoinFuncExpr = null;
+        // Maintain conditions which is not spatial_intersect in the join condition
+        List<Mutable<ILogicalExpression>> conditionExprs = new ArrayList<>();
+
+        if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.AND)) {
+            // Join condition contains multiple conditions along with spatial_intersect
+            List<Mutable<ILogicalExpression>> inputExprs = funcExpr.getArguments();
+            if (inputExprs.size() == 0) {
+                return false;
+            }
+
+            boolean spatialIntersectExists = false;
+            for (Mutable<ILogicalExpression> exp : inputExprs) {
+                AbstractFunctionCallExpression funcCallExp = (AbstractFunctionCallExpression) exp.getValue();
+                if (funcCallExp.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+                    spatialJoinFuncExpr = funcCallExp;
+                    spatialIntersectExists = true;
+                } else {
+                    // Retain the other conditions
+                    conditionExprs.add(exp);
+                }
+            }
+
+            if (!spatialIntersectExists) {
+                return false;
+            }
+        } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.SPATIAL_INTERSECT)) {
+            // Join condition is spatial_intersect only
+            spatialJoinFuncExpr = funcExpr;
+        } else {
+            return false;
+        }
+
+        // Apply the PBSM join algorithm with/without hint
+        SpatialJoinAnnotation spatialJoinAnn = spatialJoinFuncExpr.getAnnotation(SpatialJoinAnnotation.class);
+        return SpatialJoinUtils.updateJoinPlan(op, spatialJoinFuncExpr, conditionExprs, spatialJoinAnn, context, left,
+                right);
+    }
+
+    private static void setSpatialJoinOp(AbstractBinaryJoinOperator op, List<LogicalVariable> keysLeftBranch,
+            List<LogicalVariable> keysRightBranch, IOptimizationContext context) throws AlgebricksException {
+        ISpatialJoinUtilFactory isjuf = new IntersectSpatialJoinUtilFactory();
+        op.setPhysicalOperator(new SpatialJoinPOperator(op.getJoinKind(),
+                AbstractJoinPOperator.JoinPartitioningType.PAIRWISE, keysLeftBranch, keysRightBranch,
+                context.getPhysicalOptimizationConfig().getMaxFramesForJoin(), isjuf));
+        op.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(op);
+    }
+
+    private static LogicalVariable injectSpatialTileUnnestOperator(IOptimizationContext context,
+            Mutable<ILogicalOperator> op, LogicalVariable unnestVar, Mutable<ILogicalExpression> unnestMBRExpr,
+            int numRows, int numColumns) throws AlgebricksException {
+        SourceLocation srcLoc = op.getValue().getSourceLocation();
+        LogicalVariable tileIdVar = context.newVar();
+        VariableReferenceExpression unnestVarRef = new VariableReferenceExpression(unnestVar);
+        unnestVarRef.setSourceLocation(srcLoc);
+        UnnestingFunctionCallExpression spatialTileFuncExpr = new UnnestingFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.SPATIAL_TILE),
+                new MutableObject<>(unnestVarRef), unnestMBRExpr,
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numRows)))),
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numColumns)))));
+        spatialTileFuncExpr.setSourceLocation(srcLoc);
+        UnnestOperator unnestOp = new UnnestOperator(tileIdVar, new MutableObject<>(spatialTileFuncExpr));
+        unnestOp.setPhysicalOperator(new UnnestPOperator());
+        unnestOp.setSourceLocation(srcLoc);
+        unnestOp.getInputs().add(new MutableObject<>(op.getValue()));
+        context.computeAndSetTypeEnvironmentForOperator(unnestOp);
+        unnestOp.recomputeSchema();
+        op.setValue(unnestOp);
+
+        return tileIdVar;
+    }
+
+    protected static boolean updateJoinPlan(AbstractBinaryJoinOperator op,
+            AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+            SpatialJoinAnnotation spatialJoinAnn, IOptimizationContext context, int left, int right)
+            throws AlgebricksException {
+        // Extracts spatial intersect function's arguments
+        List<Mutable<ILogicalExpression>> spatialJoinArgs = spatialJoinFuncExpr.getArguments();
+        if (spatialJoinArgs.size() != 2) {
+            return false;
+        }
+
+        ILogicalExpression spatialJoinLeftArg = spatialJoinArgs.get(left).getValue();
+        ILogicalExpression spatialJoinRightArg = spatialJoinArgs.get(right).getValue();
+
+        // Left and right arguments of the spatial_intersect function should be variables
+        if (spatialJoinLeftArg.getExpressionTag() != LogicalExpressionTag.VARIABLE
+                || spatialJoinRightArg.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+            return false;
+        }
+
+        // We only apply this rule if the arguments of spatial_intersect are ARectangle
+        IVariableTypeEnvironment typeEnvironment = op.computeInputTypeEnvironment(context);
+        IAType leftType = (IAType) context.getExpressionTypeComputer().getType(spatialJoinLeftArg,
+                context.getMetadataProvider(), typeEnvironment);
+        IAType rightType = (IAType) context.getExpressionTypeComputer().getType(spatialJoinRightArg,
+                context.getMetadataProvider(), typeEnvironment);
+        if ((leftType != BuiltinType.ARECTANGLE) || (rightType != BuiltinType.ARECTANGLE)) {
+            return false;
+        }
+
+        // Gets both input branches of the spatial join.
+        Mutable<ILogicalOperator> leftInputOp = op.getInputs().get(left);
+        Mutable<ILogicalOperator> rightInputOp = op.getInputs().get(right);
+
+        // Extract left and right variable of the predicate
+        LogicalVariable spatialJoinVar0 = ((VariableReferenceExpression) spatialJoinLeftArg).getVariableReference();
+        LogicalVariable spatialJoinVar1 = ((VariableReferenceExpression) spatialJoinRightArg).getVariableReference();
+
+        LogicalVariable leftInputVar;
+        LogicalVariable rightInputVar;
+        Collection<LogicalVariable> liveVars = new HashSet<>();
+        VariableUtilities.getLiveVariables(leftInputOp.getValue(), liveVars);
+        if (liveVars.contains(spatialJoinVar0)) {
+            leftInputVar = spatialJoinVar0;
+            rightInputVar = spatialJoinVar1;
+        } else {
+            leftInputVar = spatialJoinVar1;
+            rightInputVar = spatialJoinVar0;
+        }
+
+        // If the hint is not provided, the intersection MBR of two inputs will be computed on the run time
+        if (spatialJoinAnn == null) {
+            buildSpatialJoinPlanWithDynamicMbr(op, context, spatialJoinFuncExpr, conditionExprs, leftInputOp,
+                    rightInputOp, leftInputVar, rightInputVar);
+        } else {
+            buildSpatialJoinPlanWithStaticMbr(op, context, spatialJoinFuncExpr, conditionExprs, leftInputOp,
+                    rightInputOp, leftInputVar, rightInputVar, spatialJoinAnn);
+        }
+
+        return true;
+    }
+
+    private static void buildSpatialJoinPlanWithStaticMbr(AbstractBinaryJoinOperator op, IOptimizationContext context,
+            AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+            Mutable<ILogicalOperator> leftInputOp, Mutable<ILogicalOperator> rightInputOp, LogicalVariable leftInputVar,
+            LogicalVariable rightInputVar, SpatialJoinAnnotation spatialJoinAnn) throws AlgebricksException {
+        Mutable<ILogicalExpression> leftIntersectionMBRExpr = createRectangleExpression(spatialJoinAnn);
+        Mutable<ILogicalExpression> rightIntersectionMBRExpr = createRectangleExpression(spatialJoinAnn);
+        Mutable<ILogicalExpression> referencePointTestMBRExpr = createRectangleExpression(spatialJoinAnn);
+        int numRows = spatialJoinAnn.getNumRows();
+        int numColumns = spatialJoinAnn.getNumColumns();
+
+        // Inject unnest operator to add tile ID to the left and right branch of the join operator
+        LogicalVariable leftTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, leftInputOp,
+                leftInputVar, leftIntersectionMBRExpr, numRows, numColumns);
+        LogicalVariable rightTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, rightInputOp,
+                rightInputVar, rightIntersectionMBRExpr, numRows, numColumns);
+
+        // The reference point test condition is considered as a part of spatial join conditions if a hint is provided.
+        ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+                createReferencePointTestCondition(op, referencePointTestMBRExpr, leftTileIdVar, rightTileIdVar,
+                        leftInputVar, rightInputVar, numRows, numColumns);
+        conditionExprs.add(new MutableObject<>(referenceIdEquiJoinCondition));
+
+        conditionExprs.add(new MutableObject<>(spatialJoinFuncExpr));
+
+        ScalarFunctionCallExpression updatedJoinCondition = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND), conditionExprs);
+        updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+        Mutable<ILogicalExpression> joinConditionRef = op.getCondition();
+        joinConditionRef.setValue(updatedJoinCondition);
+
+        List<LogicalVariable> keysLeftBranch = new ArrayList<>();
+        keysLeftBranch.add(leftTileIdVar);
+        keysLeftBranch.add(leftInputVar);
+
+        List<LogicalVariable> keysRightBranch = new ArrayList<>();
+        keysRightBranch.add(rightTileIdVar);
+        keysRightBranch.add(rightInputVar);
+
+        SpatialJoinUtils.setSpatialJoinOp(op, keysLeftBranch, keysRightBranch, context);
+    }
+
+    private static void buildSpatialJoinPlanWithDynamicMbr(AbstractBinaryJoinOperator op, IOptimizationContext context,
+            AbstractFunctionCallExpression spatialJoinFuncExpr, List<Mutable<ILogicalExpression>> conditionExprs,
+            Mutable<ILogicalOperator> leftInputOp, Mutable<ILogicalOperator> rightInputOp, LogicalVariable leftInputVar,
+            LogicalVariable rightInputVar) throws AlgebricksException {
+        // Add a dynamic workflow to compute MBR of the left branch
+        Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> leftMBRCalculator =
+                createDynamicMBRCalculator(op, context, leftInputOp, leftInputVar);
+        MutableObject<ILogicalOperator> leftGlobalAgg = leftMBRCalculator.first;
+        List<LogicalVariable> leftGlobalAggResultVars = leftMBRCalculator.second;
+        MutableObject<ILogicalOperator> leftExchToJoinOpRef = leftMBRCalculator.third;
+        LogicalVariable leftMBRVar = leftGlobalAggResultVars.get(0);
+
+        // Add a dynamic workflow to compute MBR of the right branch
+        Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> rightMBRCalculator =
+                createDynamicMBRCalculator(op, context, rightInputOp, rightInputVar);
+        MutableObject<ILogicalOperator> rightGlobalAgg = rightMBRCalculator.first;
+        List<LogicalVariable> rightGlobalAggResultVars = rightMBRCalculator.second;
+        MutableObject<ILogicalOperator> rightExchToJoinOpRef = rightMBRCalculator.third;
+        LogicalVariable rightMBRVar = rightGlobalAggResultVars.get(0);
+
+        // Join the left and right union MBR
+        Mutable<ILogicalExpression> trueCondition =
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+        InnerJoinOperator unionMBRJoinOp = new InnerJoinOperator(trueCondition, leftGlobalAgg, rightGlobalAgg);
+        unionMBRJoinOp.setSourceLocation(op.getSourceLocation());
+        unionMBRJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+                AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+        MutableObject<ILogicalOperator> unionMBRJoinOpRef = new MutableObject<>(unionMBRJoinOp);
+        unionMBRJoinOp.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(unionMBRJoinOp);
+
+        // Compute the intersection rectangle of left MBR and right MBR
+        List<Mutable<ILogicalExpression>> getIntersectionFuncInputExprs = new ArrayList<>();
+        getIntersectionFuncInputExprs.add(new MutableObject<>(new VariableReferenceExpression(leftMBRVar)));
+        getIntersectionFuncInputExprs.add(new MutableObject<>(new VariableReferenceExpression(rightMBRVar)));
+        ScalarFunctionCallExpression getIntersectionFuncExpr = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.GET_INTERSECTION),
+                getIntersectionFuncInputExprs);
+        getIntersectionFuncExpr.setSourceLocation(op.getSourceLocation());
+
+        Mutable<ILogicalExpression> intersectionMBRExpr = new MutableObject<>(getIntersectionFuncExpr);
+        LogicalVariable intersectionMBR = context.newVar();
+        AbstractLogicalOperator intersectionMBRAssignOperator =
+                new AssignOperator(intersectionMBR, intersectionMBRExpr);
+        intersectionMBRAssignOperator.setSourceLocation(op.getSourceLocation());
+        intersectionMBRAssignOperator.setExecutionMode(op.getExecutionMode());
+        intersectionMBRAssignOperator.setPhysicalOperator(new AssignPOperator());
+        intersectionMBRAssignOperator.getInputs().add(new MutableObject<>(unionMBRJoinOpRef.getValue()));
+        context.computeAndSetTypeEnvironmentForOperator(intersectionMBRAssignOperator);
+        intersectionMBRAssignOperator.recomputeSchema();
+        MutableObject<ILogicalOperator> intersectionMBRAssignOperatorRef =
+                new MutableObject<>(intersectionMBRAssignOperator);
+
+        // Replicate the union MBR to left and right nested loop join(NLJ) operator, and another NLJ for reference point test
+        ReplicateOperator intersectionMBRReplicateOperator =
+                createReplicateOperator(intersectionMBRAssignOperatorRef, context, op.getSourceLocation(), 3);
+
+        // Replicate union MBR to the left branch
+        ExchangeOperator exchMBRToJoinOpLeft =
+                createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+        MutableObject<ILogicalOperator> exchMBRToJoinOpLeftRef = new MutableObject<>(exchMBRToJoinOpLeft);
+        Pair<LogicalVariable, Mutable<ILogicalOperator>> createLeftAssignProjectOperatorResult =
+                createAssignProjectOperator(op, intersectionMBR, intersectionMBRReplicateOperator,
+                        exchMBRToJoinOpLeftRef, context);
+        LogicalVariable leftIntersectionMBRVar = createLeftAssignProjectOperatorResult.getFirst();
+        Mutable<ILogicalOperator> leftIntersectionMBRRef = createLeftAssignProjectOperatorResult.getSecond();
+
+        // Replicate union MBR to the right branch
+        ExchangeOperator exchMBRToJoinOpRight =
+                createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+        MutableObject<ILogicalOperator> exchMBRToJoinOpRightRef = new MutableObject<>(exchMBRToJoinOpRight);
+        Pair<LogicalVariable, Mutable<ILogicalOperator>> createRightAssignProjectOperatorResult =
+                createAssignProjectOperator(op, intersectionMBR, intersectionMBRReplicateOperator,
+                        exchMBRToJoinOpRightRef, context);
+        LogicalVariable rightIntersectionMBRVar = createRightAssignProjectOperatorResult.getFirst();
+        Mutable<ILogicalOperator> rightIntersectionMBRRef = createRightAssignProjectOperatorResult.getSecond();
+
+        // Replicate union MBR to the right branch of a later Nested Loop Join reference point test
+        ExchangeOperator exchMBRToReferencePointTestJoinOp =
+                createBroadcastExchangeOp(intersectionMBRReplicateOperator, context, op.getSourceLocation());
+        MutableObject<ILogicalOperator> exchMBRToReferencePointTestJoinOpRef =
+                new MutableObject<>(exchMBRToReferencePointTestJoinOp);
+
+        // Add left Join (TRUE)
+        Mutable<ILogicalExpression> leftTrueCondition =
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+        InnerJoinOperator leftJoinOp =
+                new InnerJoinOperator(leftTrueCondition, leftExchToJoinOpRef, leftIntersectionMBRRef);
+        leftJoinOp.setSourceLocation(op.getSourceLocation());
+        leftJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+                AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+        MutableObject<ILogicalOperator> leftJoinRef = new MutableObject<>(leftJoinOp);
+        leftJoinOp.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(leftJoinOp);
+        leftInputOp.setValue(leftJoinRef.getValue());
+
+        // Add right Join (TRUE)
+        Mutable<ILogicalExpression> rightTrueCondition =
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE)));
+        InnerJoinOperator rightJoinOp =
+                new InnerJoinOperator(rightTrueCondition, rightExchToJoinOpRef, rightIntersectionMBRRef);
+        rightJoinOp.setSourceLocation(op.getSourceLocation());
+        rightJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(AbstractBinaryJoinOperator.JoinKind.INNER,
+                AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+        MutableObject<ILogicalOperator> rightJoinRef = new MutableObject<>(rightJoinOp);
+        rightJoinOp.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(rightJoinOp);
+        rightInputOp.setValue(rightJoinRef.getValue());
+
+        Mutable<ILogicalExpression> leftIntersectionMBRExpr =
+                new MutableObject<>(new VariableReferenceExpression(leftIntersectionMBRVar));
+        Mutable<ILogicalExpression> rightIntersectionMBRExpr =
+                new MutableObject<>(new VariableReferenceExpression(rightIntersectionMBRVar));
+        Mutable<ILogicalExpression> referencePointTestMBRExpr =
+                new MutableObject<>(new VariableReferenceExpression(intersectionMBR));
+
+        // Inject unnest operator to add tile ID to the left and right branch of the join operator
+        LogicalVariable leftTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, leftInputOp,
+                leftInputVar, leftIntersectionMBRExpr, DEFAULT_ROWS, DEFAULT_COLUMNS);
+        LogicalVariable rightTileIdVar = SpatialJoinUtils.injectSpatialTileUnnestOperator(context, rightInputOp,
+                rightInputVar, rightIntersectionMBRExpr, DEFAULT_ROWS, DEFAULT_COLUMNS);
+
+        // Reference point test condition will be used as the condition of a Nested Loop Join operator after the
+        // spatial join operator. This design allow us to use the union MBR (or summary of the join) efficiently,
+        // instead of propagate this variable via Hyracks context or data flow.
+        ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+                createReferencePointTestCondition(op, referencePointTestMBRExpr, leftTileIdVar, rightTileIdVar,
+                        leftInputVar, rightInputVar, DEFAULT_ROWS, DEFAULT_COLUMNS);
+
+        conditionExprs.add(new MutableObject<>(spatialJoinFuncExpr));
+
+        ScalarFunctionCallExpression updatedJoinCondition;
+        if (conditionExprs.size() > 1) {
+            updatedJoinCondition = new ScalarFunctionCallExpression(
+                    BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND), conditionExprs);
+            updatedJoinCondition.setSourceLocation(op.getSourceLocation());
+        } else {
+            updatedJoinCondition = (ScalarFunctionCallExpression) spatialJoinFuncExpr;
+        }
+        Mutable<ILogicalExpression> joinConditionRef = op.getCondition();
+        joinConditionRef.setValue(updatedJoinCondition);
+
+        List<LogicalVariable> keysLeftBranch = new ArrayList<>();
+        keysLeftBranch.add(leftTileIdVar);
+        keysLeftBranch.add(leftInputVar);
+
+        List<LogicalVariable> keysRightBranch = new ArrayList<>();
+        keysRightBranch.add(rightTileIdVar);
+        keysRightBranch.add(rightInputVar);
+
+        InnerJoinOperator spatialJoinOp =
+                new InnerJoinOperator(new MutableObject<>(updatedJoinCondition), leftInputOp, rightInputOp);
+        spatialJoinOp.setSourceLocation(op.getSourceLocation());
+        SpatialJoinUtils.setSpatialJoinOp(spatialJoinOp, keysLeftBranch, keysRightBranch, context);
+        spatialJoinOp.setSchema(op.getSchema());
+        context.computeAndSetTypeEnvironmentForOperator(spatialJoinOp);
+
+        Mutable<ILogicalOperator> opRef = new MutableObject<>(op);
+        Mutable<ILogicalOperator> spatialJoinOpRef = new MutableObject<>(spatialJoinOp);
+
+        InnerJoinOperator referencePointTestJoinOp =
+                new InnerJoinOperator(new MutableObject<>(referenceIdEquiJoinCondition), spatialJoinOpRef,
+                        exchMBRToReferencePointTestJoinOpRef);
+        referencePointTestJoinOp.setPhysicalOperator(new NestedLoopJoinPOperator(
+                AbstractBinaryJoinOperator.JoinKind.INNER, AbstractJoinPOperator.JoinPartitioningType.BROADCAST));
+        MutableObject<ILogicalOperator> referencePointTestJoinOpRef = new MutableObject<>(referencePointTestJoinOp);
+        referencePointTestJoinOp.setSourceLocation(op.getSourceLocation());
+        context.computeAndSetTypeEnvironmentForOperator(referencePointTestJoinOp);
+        referencePointTestJoinOp.recomputeSchema();
+        opRef.setValue(referencePointTestJoinOpRef.getValue());
+        op.getInputs().clear();
+        op.getInputs().addAll(referencePointTestJoinOp.getInputs());
+        op.setPhysicalOperator(referencePointTestJoinOp.getPhysicalOperator());
+        op.getCondition().setValue(referencePointTestJoinOp.getCondition().getValue());
+        context.computeAndSetTypeEnvironmentForOperator(op);
+        op.recomputeSchema();
+    }
+
+    private static ScalarFunctionCallExpression createReferencePointTestCondition(AbstractBinaryJoinOperator op,
+            Mutable<ILogicalExpression> referencePointTestMBRExpr, LogicalVariable leftTileIdVar,
+            LogicalVariable rightTileIdVar, LogicalVariable leftInputVar, LogicalVariable rightInputVar, int numRows,
+            int numColumns) {
+        // Compute reference tile ID
+        ScalarFunctionCallExpression referenceTileId = new ScalarFunctionCallExpression(
+                BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.REFERENCE_TILE),
+                new MutableObject<>(new VariableReferenceExpression(leftInputVar)),
+                new MutableObject<>(new VariableReferenceExpression(rightInputVar)), referencePointTestMBRExpr,
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numRows)))),
+                new MutableObject<>(new ConstantExpression(new AsterixConstantValue(new AInt64(numColumns)))),
+                new MutableObject<>(new VariableReferenceExpression(rightTileIdVar)));
+        referenceTileId.setSourceLocation(op.getSourceLocation());
+
+        ScalarFunctionCallExpression referenceIdEquiJoinCondition =
+                new ScalarFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.EQ),
+                        new MutableObject<>(new VariableReferenceExpression(leftTileIdVar)),
+                        new MutableObject<>(referenceTileId));
+        referenceIdEquiJoinCondition.setSourceLocation(op.getSourceLocation());
+
+        return referenceIdEquiJoinCondition;
+    }
+
+    private static Pair<LogicalVariable, Mutable<ILogicalOperator>> createAssignProjectOperator(
+            AbstractBinaryJoinOperator op, LogicalVariable inputVar, ReplicateOperator replicateOperator,
+            MutableObject<ILogicalOperator> exchMBRToForwardRef, IOptimizationContext context)
+            throws AlgebricksException {
+        LogicalVariable newFinalMbrVar = context.newVar();
+        List<LogicalVariable> finalMBRLiveVars = new ArrayList<>();
+        finalMBRLiveVars.add(newFinalMbrVar);
+        ListSet<LogicalVariable> finalMBRLiveVarsSet = new ListSet<>();
+        finalMBRLiveVarsSet.add(newFinalMbrVar);
+
+        Mutable<ILogicalExpression> finalMBRExpr = new MutableObject<>(new VariableReferenceExpression(inputVar));
+        AbstractLogicalOperator assignOperator = new AssignOperator(newFinalMbrVar, finalMBRExpr);
+        assignOperator.setSourceLocation(op.getSourceLocation());
+        assignOperator.setExecutionMode(replicateOperator.getExecutionMode());
+        assignOperator.setPhysicalOperator(new AssignPOperator());
+        AbstractLogicalOperator projectOperator = new ProjectOperator(finalMBRLiveVars);
+        projectOperator.setSourceLocation(op.getSourceLocation());
+        projectOperator.setPhysicalOperator(new StreamProjectPOperator());
+        projectOperator.setExecutionMode(replicateOperator.getExecutionMode());
+        assignOperator.getInputs().add(exchMBRToForwardRef);
+        projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(assignOperator));
+
+        context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+        assignOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(projectOperator);
+        projectOperator.recomputeSchema();
+        Mutable<ILogicalOperator> projectOperatorRef = new MutableObject<>(projectOperator);
+
+        return new Pair<>(newFinalMbrVar, projectOperatorRef);
+    }
+
+    private static ReplicateOperator createReplicateOperator(Mutable<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation, int outputArity) throws AlgebricksException {
+        ReplicateOperator replicateOperator = new ReplicateOperator(outputArity);
+        replicateOperator.setPhysicalOperator(new ReplicatePOperator());
+        replicateOperator.setSourceLocation(sourceLocation);
+        replicateOperator.getInputs().add(new MutableObject<>(inputOperator.getValue()));
+        OperatorManipulationUtil.setOperatorMode(replicateOperator);
+        replicateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(replicateOperator);
+        return replicateOperator;
+    }
+
+    private static ExchangeOperator createOneToOneExchangeOp(ReplicateOperator replicateOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        ExchangeOperator exchangeOperator = new ExchangeOperator();
+        exchangeOperator.setSourceLocation(sourceLocation);
+        exchangeOperator.setPhysicalOperator(new OneToOneExchangePOperator());
+        replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+        exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+        exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        exchangeOperator.setSchema(replicateOperator.getSchema());
+        context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+        return exchangeOperator;
+    }
+
+    private static ExchangeOperator createRandomPartitionExchangeOp(ReplicateOperator replicateOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        ExchangeOperator exchangeOperator = new ExchangeOperator();
+        exchangeOperator.setSourceLocation(sourceLocation);
+        exchangeOperator.setPhysicalOperator(new RandomPartitionExchangePOperator(context.getComputationNodeDomain()));
+        replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+        exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+        exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        exchangeOperator.setSchema(replicateOperator.getSchema());
+        context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+        return exchangeOperator;
+    }
+
+    private static ExchangeOperator createBroadcastExchangeOp(ReplicateOperator replicateOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        ExchangeOperator exchangeOperator = new ExchangeOperator();
+        exchangeOperator.setSourceLocation(sourceLocation);
+        exchangeOperator.setPhysicalOperator(new BroadcastExchangePOperator(context.getComputationNodeDomain()));
+        replicateOperator.getOutputs().add(new MutableObject<>(exchangeOperator));
+        exchangeOperator.getInputs().add(new MutableObject<>(replicateOperator));
+        exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        exchangeOperator.setSchema(replicateOperator.getSchema());
+        context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+        return exchangeOperator;
+    }
+
+    private static Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createLocalAndGlobalAggregateOperators(
+            AbstractBinaryJoinOperator op, IOptimizationContext context, LogicalVariable inputVar,
+            MutableObject<ILogicalOperator> exchToLocalAggRef) throws AlgebricksException {
+        AbstractLogicalExpression inputVarRef = new VariableReferenceExpression(inputVar, op.getSourceLocation());
+        List<Mutable<ILogicalExpression>> fields = new ArrayList<>(1);
+        fields.add(new MutableObject<>(inputVarRef));
+
+        // Create local aggregate operator
+        IFunctionInfo localAggFunc = context.getMetadataProvider().lookupFunction(BuiltinFunctions.LOCAL_UNION_MBR);
+        AggregateFunctionCallExpression localAggExpr = new AggregateFunctionCallExpression(localAggFunc, false, fields);
+        localAggExpr.setSourceLocation(op.getSourceLocation());
+        localAggExpr.setOpaqueParameters(new Object[] {});
+        List<LogicalVariable> localAggResultVars = new ArrayList<>(1);
+        List<Mutable<ILogicalExpression>> localAggFuncs = new ArrayList<>(1);
+        LogicalVariable localOutVariable = context.newVar();
+        localAggResultVars.add(localOutVariable);
+        localAggFuncs.add(new MutableObject<>(localAggExpr));
+        AggregateOperator localAggOperator = createAggregate(localAggResultVars, false, localAggFuncs,
+                exchToLocalAggRef, context, op.getSourceLocation());
+        MutableObject<ILogicalOperator> localAgg = new MutableObject<>(localAggOperator);
+
+        // Output of local aggregate operator is the input of global aggregate operator
+        return createGlobalAggregateOperator(op, context, localOutVariable, localAgg);
+    }
+
+    private static Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createGlobalAggregateOperator(
+            AbstractBinaryJoinOperator op, IOptimizationContext context, LogicalVariable inputVar,
+            MutableObject<ILogicalOperator> inputOperator) throws AlgebricksException {
+        List<Mutable<ILogicalExpression>> globalAggFuncArgs = new ArrayList<>(1);
+        AbstractLogicalExpression inputVarRef = new VariableReferenceExpression(inputVar, op.getSourceLocation());
+        globalAggFuncArgs.add(new MutableObject<>(inputVarRef));
+        IFunctionInfo globalAggFunc = context.getMetadataProvider().lookupFunction(BuiltinFunctions.GLOBAL_UNION_MBR);
+        AggregateFunctionCallExpression globalAggExpr =
+                new AggregateFunctionCallExpression(globalAggFunc, true, globalAggFuncArgs);
+        globalAggExpr.setStepOneAggregate(globalAggFunc);
+        globalAggExpr.setStepTwoAggregate(globalAggFunc);
+        globalAggExpr.setSourceLocation(op.getSourceLocation());
+        globalAggExpr.setOpaqueParameters(new Object[] {});
+        List<LogicalVariable> globalAggResultVars = new ArrayList<>(1);
+        globalAggResultVars.add(context.newVar());
+        List<Mutable<ILogicalExpression>> globalAggFuncs = new ArrayList<>(1);
+        globalAggFuncs.add(new MutableObject<>(globalAggExpr));
+        AggregateOperator globalAggOperator = createAggregate(globalAggResultVars, true, globalAggFuncs, inputOperator,
+                context, op.getSourceLocation());
+        globalAggOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(globalAggOperator);
+        MutableObject<ILogicalOperator> globalAgg = new MutableObject<>(globalAggOperator);
+        return new Pair<>(globalAgg, globalAggResultVars);
+    }
+
+    private static Triple<MutableObject<ILogicalOperator>, List<LogicalVariable>, MutableObject<ILogicalOperator>> createDynamicMBRCalculator(
+            AbstractBinaryJoinOperator op, IOptimizationContext context, Mutable<ILogicalOperator> inputOp,
+            LogicalVariable inputVar) throws AlgebricksException {
+        // Add ReplicationOperator for the input branch
+        SourceLocation sourceLocation = op.getSourceLocation();
+        ReplicateOperator replicateOperator = createReplicateOperator(inputOp, context, sourceLocation, 2);
+
+        // Create one to one exchange operators for the replicator of the input branch
+        ExchangeOperator exchToForward = createRandomPartitionExchangeOp(replicateOperator, context, sourceLocation);
+        MutableObject<ILogicalOperator> exchToForwardRef = new MutableObject<>(exchToForward);
+
+        ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(replicateOperator, context, op.getSourceLocation());
+        MutableObject<ILogicalOperator> exchToLocalAggRef = new MutableObject<>(exchToLocalAgg);
+
+        // Materialize the data to be able to re-read the data again
+        replicateOperator.getOutputMaterializationFlags()[0] = true;
+
+        Pair<MutableObject<ILogicalOperator>, List<LogicalVariable>> createLocalAndGlobalAggResult =
+                createLocalAndGlobalAggregateOperators(op, context, inputVar, exchToLocalAggRef);
+        return new Triple<>(createLocalAndGlobalAggResult.first, createLocalAndGlobalAggResult.second,
+                exchToForwardRef);
+    }
+
+    /**
+     * Creates an aggregate operator. $$resultVariables = expressions()
+     * @param resultVariables the variables which stores the result of the aggregation
+     * @param isGlobal whether the aggregate operator is a global or local one
+     * @param expressions the aggregation functions desired
+     * @param inputOperator the input op that is feeding the aggregate operator
+     * @param context optimization context
+     * @param sourceLocation source location
+     * @return an aggregate operator with the specified information
+     * @throws AlgebricksException when there is error setting the type environment of the newly created aggregate op
+     */
+    private static AggregateOperator createAggregate(List<LogicalVariable> resultVariables, boolean isGlobal,
+            List<Mutable<ILogicalExpression>> expressions, MutableObject<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        AggregateOperator aggregateOperator = new AggregateOperator(resultVariables, expressions);
+        aggregateOperator.setPhysicalOperator(new AggregatePOperator());
+        aggregateOperator.setSourceLocation(sourceLocation);
+        aggregateOperator.getInputs().add(inputOperator);
+        aggregateOperator.setGlobal(isGlobal);
+        if (!isGlobal) {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+        } else {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        }
+        aggregateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+        return aggregateOperator;
+    }
+
+    private static Mutable<ILogicalExpression> createRectangleExpression(SpatialJoinAnnotation spatialJoinAnn) {
+        return new MutableObject<>(new ConstantExpression(
+                new AsterixConstantValue(new ARectangle(new APoint(spatialJoinAnn.getMinX(), spatialJoinAnn.getMinY()),
+                        new APoint(spatialJoinAnn.getMaxX(), spatialJoinAnn.getMaxY())))));
+    }
+}
diff --git a/asterixdb/asterix-app/data/spatial/lakes.json b/asterixdb/asterix-app/data/spatial/lakes.json
new file mode 100644
index 0000000..0e7e4e2
--- /dev/null
+++ b/asterixdb/asterix-app/data/spatial/lakes.json
@@ -0,0 +1,100 @@
+{"id":0,"geom":rectangle("-27.9323481511,9.15518656723 -24.8445930928,11.9309972473")}
+{"id":1,"geom":rectangle("0.0621153624124,18.2734730769 4.83281963665,19.3336733634")}
+{"id":2,"geom":rectangle("-37.873318599,50.4137494117 -33.4774896319,55.3172948124")}
+{"id":3,"geom":rectangle("47.1325883602,-19.4791980415 50.9598502084,-18.481896624")}
+{"id":4,"geom":rectangle("51.3016231532,-5.60195086869 55.9768771092,-4.44676667106")}
+{"id":5,"geom":rectangle("9.27570071324,-1.68430260794 14.3618786871,0.0989231691509")}
+{"id":6,"geom":rectangle("-40.8426642141,-12.3828594111 -36.2560812836,-8.69636838084")}
+{"id":7,"geom":rectangle("42.0000180351,-6.38480480539 46.3487445182,-2.08245163105")}
+{"id":8,"geom":rectangle("-49.1105328926,1.23797277776 -48.6903344422,2.26860151234")}
+{"id":9,"geom":rectangle("8.41156431616,23.7284234749 12.3270757028,25.9675336727")}
+{"id":10,"geom":rectangle("24.2568615633,-20.3953636207 27.8001525267,-16.0145481861")}
+{"id":11,"geom":rectangle("-16.3094782394,8.91257038501 -12.3410467932,13.799767852")}
+{"id":12,"geom":rectangle("-5.47983206871,10.2243601629 -5.10262146508,13.9002669231")}
+{"id":13,"geom":rectangle("-60.9461639943,10.4323876808 -56.7511118231,14.8796053416")}
+{"id":14,"geom":rectangle("-11.972629563,15.8894382199 -8.06809484334,18.6230609701")}
+{"id":15,"geom":rectangle("-55.4148350065,-11.2390940595 -51.7126250696,-7.70494356838")}
+{"id":16,"geom":rectangle("-39.6238055795,5.95626957878 -37.3601098467,6.89396261843")}
+{"id":17,"geom":rectangle("-9.31124052127,-11.7939905005 -5.81616506883,-10.7313254896")}
+{"id":18,"geom":rectangle("55.8575494942,-34.8735358882 59.052052825,-32.1878431332")}
+{"id":19,"geom":rectangle("6.78644433431,-15.1103391491 11.022852731,-9.97257905438")}
+{"id":20,"geom":rectangle("-46.9172489187,8.34059859105 -44.8843063877,10.2219856068")}
+{"id":21,"geom":rectangle("18.8133717489,7.58225769546 20.7327951646,10.400196251")}
+{"id":22,"geom":rectangle("39.6991218048,-19.5111127309 43.4813236282,-17.2751694545")}
+{"id":23,"geom":rectangle("-53.0916999826,15.6224380883 -51.3276055144,18.4131633566")}
+{"id":24,"geom":rectangle("15.4403413661,-16.717177507 20.5659895946,-14.4905598285")}
+{"id":25,"geom":rectangle("18.0403882995,-16.942547031 18.4674639813,-12.0137417407")}
+{"id":26,"geom":rectangle("-10.1680547254,-60.7865500782 -8.00879668415,-56.7438699052")}
+{"id":27,"geom":rectangle("68.6004972482,8.11792475815 71.5635512937,11.1498717613")}
+{"id":28,"geom":rectangle("-42.941592897,-12.1682044962 -37.7633402924,-11.336014273")}
+{"id":29,"geom":rectangle("72.6533873046,2.07227812745 76.6403854525,7.09220721646")}
+{"id":30,"geom":rectangle("51.8540534407,-17.0442271016 54.8389508618,-12.6231227706")}
+{"id":31,"geom":rectangle("6.60620427018,15.9937496153 8.0217849552,16.8439308667")}
+{"id":32,"geom":rectangle("-54.4890021697,-23.8103029028 -50.031768851,-22.2413347517")}
+{"id":33,"geom":rectangle("-34.95511358,2.02954068386 -33.703431683,4.19422929144")}
+{"id":34,"geom":rectangle("-20.7743277107,-0.496422564668 -19.777600253,-0.0970785280508")}
+{"id":35,"geom":rectangle("-39.3142027252,-3.48265518567 -38.0781722188,-1.12776007008")}
+{"id":36,"geom":rectangle("6.77600057461,26.8759104261 11.7404611175,32.0174621982")}
+{"id":37,"geom":rectangle("24.4099866125,3.57651968637 26.7640691754,4.88288221349")}
+{"id":38,"geom":rectangle("15.2975621765,-0.569018270344 17.3906547082,1.74652310434")}
+{"id":39,"geom":rectangle("56.9872441628,-2.94723993425 61.3324446299,0.926328661561")}
+{"id":40,"geom":rectangle("41.047461317,10.1632058932 44.7623919157,13.6693470556")}
+{"id":41,"geom":rectangle("10.1079158628,-23.8702911151 12.7244666464,-22.8542384108")}
+{"id":42,"geom":rectangle("-22.4974254873,11.8509722536 -21.8574961025,15.2160233895")}
+{"id":43,"geom":rectangle("-43.9972059177,25.2595980469 -43.2811239229,28.5826167355")}
+{"id":44,"geom":rectangle("-7.23037396361,-7.11084459626 -2.27629349239,-3.61114277177")}
+{"id":45,"geom":rectangle("41.7404274398,16.4641333114 42.3134948646,17.21273776")}
+{"id":46,"geom":rectangle("-78.8108223124,33.0735094389 -73.6471184423,35.1192424639")}
+{"id":47,"geom":rectangle("69.3770819027,-8.4844290646 72.8377016978,-6.7522689002")}
+{"id":48,"geom":rectangle("-12.3843945904,4.42691376249 -9.57989036317,8.73880139764")}
+{"id":49,"geom":rectangle("4.26177443767,23.7431103696 8.16629007069,27.5738476242")}
+{"id":50,"geom":rectangle("-32.8317754069,-34.9288132423 -32.5521111063,-30.0667506035")}
+{"id":51,"geom":rectangle("38.3564015651,3.76035299468 42.9897451059,6.28666914167")}
+{"id":52,"geom":rectangle("9.66830715974,-13.2164544981 13.4317818497,-9.61970001253")}
+{"id":53,"geom":rectangle("-41.8619178712,0.833642096163 -37.7009933113,4.95720796208")}
+{"id":54,"geom":rectangle("36.9003622721,-16.1326998758 38.2630992969,-14.1602245262")}
+{"id":55,"geom":rectangle("-39.2043477954,-8.37064618527 -37.5700101189,-3.7861350448")}
+{"id":56,"geom":rectangle("-73.1306562341,-28.4389790443 -72.7740930226,-25.4986564116")}
+{"id":57,"geom":rectangle("-33.9976660158,5.22879142129 -32.8492047898,10.3133401489")}
+{"id":58,"geom":rectangle("-77.1746910469,0.922084338132 -75.6504210426,1.2034103681")}
+{"id":59,"geom":rectangle("22.2027355036,24.8837157674 23.9232110479,28.4103275964")}
+{"id":60,"geom":rectangle("9.88465219327,19.6537576964 10.2765884889,20.3193773926")}
+{"id":61,"geom":rectangle("-26.8112564518,2.7690970691 -22.5926465585,6.93867080924")}
+{"id":62,"geom":rectangle("-43.7073938674,-6.60657607222 -41.4240040146,-2.71455237271")}
+{"id":63,"geom":rectangle("24.4048294711,-4.62891935142 27.906994004,-4.29135734323")}
+{"id":64,"geom":rectangle("51.6531727755,21.8910939472 52.3394468954,23.2826790089")}
+{"id":65,"geom":rectangle("25.8914076927,6.92224912861 27.3432267461,9.47019756737")}
+{"id":66,"geom":rectangle("13.3696451516,5.615604925 16.9054074708,8.52144979175")}
+{"id":67,"geom":rectangle("-13.6800980239,-18.9444336886 -9.95347951639,-16.7345361171")}
+{"id":68,"geom":rectangle("-17.7510748238,-11.1136308762 -15.412795239,-7.34469680748")}
+{"id":69,"geom":rectangle("33.2947868986,8.2834932121 34.7613269962,9.03569363267")}
+{"id":70,"geom":rectangle("-33.8874431333,-23.8338234095 -30.7254105611,-20.359752895")}
+{"id":71,"geom":rectangle("22.0907475181,2.29036800869 23.7011151494,3.2765296671")}
+{"id":72,"geom":rectangle("14.5755950289,52.8417124837 19.5671060363,55.1073390372")}
+{"id":73,"geom":rectangle("-18.8077581597,16.9934859515 -15.9861161902,17.8343474049")}
+{"id":74,"geom":rectangle("10.9084864464,29.5391931137 15.3037198201,33.1052109059")}
+{"id":75,"geom":rectangle("19.4718498204,7.79868609475 22.1664168013,12.1717440014")}
+{"id":76,"geom":rectangle("-3.62863167929,11.4665296807 -3.30540699392,14.1121656494")}
+{"id":77,"geom":rectangle("-41.64372999,-23.0555507792 -36.4832305461,-21.3068023443")}
+{"id":78,"geom":rectangle("26.5213107831,46.2022585409 30.6432878791,47.0129737813")}
+{"id":79,"geom":rectangle("10.789264065,21.5625965712 14.840984666,22.3260628176")}
+{"id":80,"geom":rectangle("10.9202625272,-19.0658990104 14.1023247437,-15.3016631473")}
+{"id":81,"geom":rectangle("-15.7551653757,-2.25551304155 -12.4037949556,2.0012711873")}
+{"id":82,"geom":rectangle("37.5942150963,10.997741867 41.372075762,16.1596730965")}
+{"id":83,"geom":rectangle("-13.7359186757,29.958869656 -11.4409182302,33.4511779239")}
+{"id":84,"geom":rectangle("-13.0283243952,-11.3762227927 -12.0713603337,-8.89907873732")}
+{"id":85,"geom":rectangle("77.5973017056,-22.6402092272 78.7497490033,-21.2308882262")}
+{"id":86,"geom":rectangle("-22.4088952378,-21.7807762792 -21.497054438,-17.9602452153")}
+{"id":87,"geom":rectangle("21.2973448736,26.9027621694 22.622096107,28.9878687289")}
+{"id":88,"geom":rectangle("2.51672920699,0.0240546639022 7.26260016522,1.81047192527")}
+{"id":89,"geom":rectangle("7.51495774499,-3.85627105055 8.35285901504,-0.886614413758")}
+{"id":90,"geom":rectangle("-9.42837494505,-3.98577929823 -4.62420944265,-2.23733766023")}
+{"id":91,"geom":rectangle("-40.7216302836,3.90553033275 -37.0193557402,5.3260889493")}
+{"id":92,"geom":rectangle("20.1823194132,-26.8902590996 23.5753973148,-24.3474326885")}
+{"id":93,"geom":rectangle("37.4808096204,-26.1731289166 39.5942876414,-24.2173822745")}
+{"id":94,"geom":rectangle("-1.0730021895,-26.6770190359 -0.0802012128098,-22.3629468705")}
+{"id":95,"geom":rectangle("-16.4351749786,-17.9132294166 -11.378109551,-17.0573428992")}
+{"id":96,"geom":rectangle("-1.88355736652,8.26818869244 -0.978125324269,9.48445568")}
+{"id":97,"geom":rectangle("-33.1347880034,11.6973366548 -31.6968872823,12.0041059694")}
+{"id":98,"geom":rectangle("-25.2878269988,10.2165628058 -23.4713285433,14.3321956802")}
+{"id":99,"geom":rectangle("56.4069666051,25.5819994228 59.1720253242,30.0069600333")}
diff --git a/asterixdb/asterix-app/data/spatial/parks.json b/asterixdb/asterix-app/data/spatial/parks.json
new file mode 100644
index 0000000..6a3db62
--- /dev/null
+++ b/asterixdb/asterix-app/data/spatial/parks.json
@@ -0,0 +1,100 @@
+{"id":0,"geom":rectangle("33.0815213198,-9.81639132361 37.8016305166,-7.84406572273")}
+{"id":1,"geom":rectangle("-124.421948157,1.93767883788 -124.154251809,2.28744269497")}
+{"id":2,"geom":rectangle("57.9915252427,-2.36135910497 62.6034311404,-1.82486897086")}
+{"id":3,"geom":rectangle("-29.0688935271,-26.5487124393 -25.497240315,-23.6620099901")}
+{"id":4,"geom":rectangle("44.5633661766,-10.0985164886 45.7269764454,-5.25971946827")}
+{"id":5,"geom":rectangle("15.305286444,-12.2753304202 18.9827351797,-9.36713711588")}
+{"id":6,"geom":rectangle("-29.911107762,22.5440353307 -27.4139067872,23.0896507211")}
+{"id":7,"geom":rectangle("-57.6425342931,-12.8580151738 -52.8507052534,-12.4825801116")}
+{"id":8,"geom":rectangle("25.8735318882,-9.38051778287 27.4904674652,-6.75725388702")}
+{"id":9,"geom":rectangle("9.84546779304,34.7908792934 11.3423158265,37.6163294751")}
+{"id":10,"geom":rectangle("-51.9726341759,10.6323648191 -50.5305531532,15.7945290881")}
+{"id":11,"geom":rectangle("-23.4907513017,20.662944916 -19.4319569217,23.4464479085")}
+{"id":12,"geom":rectangle("43.4201149814,12.3343076751 47.4451061727,12.9883840827")}
+{"id":13,"geom":rectangle("29.4059288562,5.16907601248 34.5850093403,6.2687077698")}
+{"id":14,"geom":rectangle("32.5674476303,-16.9214726806 36.3961900958,-16.2800680951")}
+{"id":15,"geom":rectangle("-0.1834488207,6.13920491381 4.72384569903,9.56598789645")}
+{"id":16,"geom":rectangle("38.7756881143,5.39841667745 42.7750262309,10.0289384536")}
+{"id":17,"geom":rectangle("-14.44056663,18.4307856816 -9.36677067147,19.7583766879")}
+{"id":18,"geom":rectangle("11.570745802,-6.80699899747 13.9937592089,-4.86081653246")}
+{"id":19,"geom":rectangle("17.5420798083,18.958729748 21.7364051367,23.8559784582")}
+{"id":20,"geom":rectangle("50.3626687343,-5.75624879038 50.7160445447,-1.90079629364")}
+{"id":21,"geom":rectangle("59.9180321974,-0.209278657599 60.3779738543,1.67329307706")}
+{"id":22,"geom":rectangle("-13.4143437329,-5.4128583532 -8.51004730147,-2.97939716072")}
+{"id":23,"geom":rectangle("-67.7749282406,34.4099425167 -62.7649400663,36.0128931629")}
+{"id":24,"geom":rectangle("-19.6954394465,-9.38097306619 -14.5717338989,-5.62158895288")}
+{"id":25,"geom":rectangle("-4.62283914627,14.1191109493 -0.582166990327,16.8665197135")}
+{"id":26,"geom":rectangle("11.9450583857,-18.3252119827 15.1481317185,-13.305454101")}
+{"id":27,"geom":rectangle("45.3300959836,15.1502443345 46.1914046026,16.5505569007")}
+{"id":28,"geom":rectangle("-38.1544150849,13.2427359229 -37.7223058052,13.9289037156")}
+{"id":29,"geom":rectangle("25.9117553379,-24.2063944594 27.2556664105,-20.0878926667")}
+{"id":30,"geom":rectangle("13.6057700863,-6.29749212816 14.2712566373,-3.79005795277")}
+{"id":31,"geom":rectangle("-26.9662410313,19.1425547919 -21.9190087014,22.6025571835")}
+{"id":32,"geom":rectangle("39.9580362433,6.31802974783 40.6550631538,7.78428466256")}
+{"id":33,"geom":rectangle("-22.1168360436,-20.6231838474 -21.0373812122,-17.4818461759")}
+{"id":34,"geom":rectangle("-13.7706153852,30.000930354 -10.2285702156,30.4633714206")}
+{"id":35,"geom":rectangle("10.7513899208,-46.332627949 15.1729042238,-41.7087522917")}
+{"id":36,"geom":rectangle("54.398317257,11.2893054229 55.5126158122,11.7291194213")}
+{"id":37,"geom":rectangle("63.5968193492,-15.1966382455 65.352375016,-11.0480131127")}
+{"id":38,"geom":rectangle("47.1514141104,-4.68389071981 50.0106551109,0.398482234144")}
+{"id":39,"geom":rectangle("-17.4590864829,-0.237667446208 -15.5534126253,0.984234392487")}
+{"id":40,"geom":rectangle("13.1160139515,-19.784760558 14.4541800302,-17.909695768")}
+{"id":41,"geom":rectangle("53.3179838434,15.3179563991 54.2805821576,17.6281387012")}
+{"id":42,"geom":rectangle("31.369457427,13.165056978 35.1955254051,15.0403301856")}
+{"id":43,"geom":rectangle("17.1741906327,22.8030636353 17.4889623468,26.5730209982")}
+{"id":44,"geom":rectangle("-10.3878457081,-0.815065604011 -9.62601719521,2.09908908899")}
+{"id":45,"geom":rectangle("-16.3825969516,-1.12018500993 -15.9860095134,2.45473855859")}
+{"id":46,"geom":rectangle("-33.1253688726,7.46964360071 -30.5439240628,9.95558524072")}
+{"id":47,"geom":rectangle("-29.368011305,11.6519597155 -28.3048883686,12.9818049726")}
+{"id":48,"geom":rectangle("35.0611437776,7.03034670814 39.8962767006,8.40495515366")}
+{"id":49,"geom":rectangle("-41.2271620731,1.73252488725 -38.6763728352,2.5739855887")}
+{"id":50,"geom":rectangle("-58.3254878558,-1.93159998918 -54.4272611956,2.80164821601")}
+{"id":51,"geom":rectangle("13.8023199965,37.9991713987 16.5154301058,41.2520828093")}
+{"id":52,"geom":rectangle("-0.00851486511255,-14.2470351009 3.64419833081,-9.47603639972")}
+{"id":53,"geom":rectangle("26.7054431235,-5.16438137653 31.3237631214,-2.54420971647")}
+{"id":54,"geom":rectangle("27.8866028806,7.75111403626 31.7478444739,12.478395325")}
+{"id":55,"geom":rectangle("8.21394812371,-2.29056972429 11.4573593773,-0.764304448463")}
+{"id":56,"geom":rectangle("8.46285788596,1.64018059958 12.2673723294,4.75102475847")}
+{"id":57,"geom":rectangle("1.55454244344,-12.6324441794 1.9419040003,-8.32756051385")}
+{"id":58,"geom":rectangle("81.9243274831,15.3399706853 82.9893838755,20.2484476347")}
+{"id":59,"geom":rectangle("-3.80521441571,5.59544653202 1.00217973611,6.45642148213")}
+{"id":60,"geom":rectangle("7.3027607876,41.7183918554 9.61818587568,42.6701949556")}
+{"id":61,"geom":rectangle("1.50687245994,-0.463553775161 1.71988336805,0.5148202732")}
+{"id":62,"geom":rectangle("8.1114994271,-16.9975459513 10.181118141,-12.4214451465")}
+{"id":63,"geom":rectangle("8.47517844885,3.94583778361 9.75608493291,6.45740496653")}
+{"id":64,"geom":rectangle("32.8142697785,1.7347153599 33.5075936393,4.03213277156")}
+{"id":65,"geom":rectangle("-13.9851977532,10.2201556852 -8.94180139582,10.8359476684")}
+{"id":66,"geom":rectangle("-5.81864220867,5.72831389433 -3.96515661333,9.40311082423")}
+{"id":67,"geom":rectangle("-5.94114472317,22.0425945223 -4.75208419142,25.1480304519")}
+{"id":68,"geom":rectangle("3.32138012085,-2.54014463571 4.09010361369,2.22061165506")}
+{"id":69,"geom":rectangle("24.8845851876,-3.56812813721 26.1424974309,-1.93911469198")}
+{"id":70,"geom":rectangle("8.86845184432,6.96043446979 10.6719210071,10.5089327467")}
+{"id":71,"geom":rectangle("-10.2928503721,13.7414792395 -6.33804173624,18.799298066")}
+{"id":72,"geom":rectangle("-39.2694219453,30.5862681062 -36.709885472,32.6290866592")}
+{"id":73,"geom":rectangle("-0.680131563271,27.8806141259 4.25320010443,28.960840374")}
+{"id":74,"geom":rectangle("-4.18305807732,20.931302603 -1.58595809374,24.800187646")}
+{"id":75,"geom":rectangle("21.8984327715,-1.33650659675 22.9297670191,0.553946005393")}
+{"id":76,"geom":rectangle("-24.1977676209,4.21509449843 -19.3574031974,5.40102847909")}
+{"id":77,"geom":rectangle("-15.4154311616,24.7153248018 -13.8319530786,29.4011183457")}
+{"id":78,"geom":rectangle("1.94643236293,50.3738409701 5.89810279711,54.5201490674")}
+{"id":79,"geom":rectangle("-7.96849116411,20.5720060343 -3.12254526065,21.5654341637")}
+{"id":80,"geom":rectangle("-21.7043763659,10.1241268849 -20.8416042867,13.1904255813")}
+{"id":81,"geom":rectangle("-23.3888414298,-7.16854549483 -19.3844982673,-4.41149238515")}
+{"id":82,"geom":rectangle("-33.4895210367,17.3253537643 -28.824401802,21.3670812131")}
+{"id":83,"geom":rectangle("-42.3546712937,9.76999085801 -37.8559413646,12.6720101498")}
+{"id":84,"geom":rectangle("38.8013417317,1.10992679636 40.3179710832,6.1153997663")}
+{"id":85,"geom":rectangle("4.96535858672,23.1457530824 10.1207790147,27.3333356925")}
+{"id":86,"geom":rectangle("-90.4574845046,-19.5212053021 -85.9504562794,-15.0951106414")}
+{"id":87,"geom":rectangle("-57.8849765268,6.34375036955 -54.4546233959,7.12447857349")}
+{"id":88,"geom":rectangle("1.00138643712,7.96484069816 2.42708795033,12.8608965948")}
+{"id":89,"geom":rectangle("14.3045039032,7.89032274379 16.1025253052,9.76043987428")}
+{"id":90,"geom":rectangle("-39.3422390328,29.6303954006 -35.0044772013,33.7255680238")}
+{"id":91,"geom":rectangle("-34.5615691092,15.0808712376 -32.7510701778,16.2024568771")}
+{"id":92,"geom":rectangle("-89.4105780877,-18.6334926022 -85.1904254639,-17.9571587502")}
+{"id":93,"geom":rectangle("-1.31609859863,-4.2409753097 3.68297237575,-1.77236714054")}
+{"id":94,"geom":rectangle("62.0933242625,16.4294408774 63.5309980156,17.6884479663")}
+{"id":95,"geom":rectangle("-12.3749081173,-14.5326618303 -12.1631207021,-9.77210703996")}
+{"id":96,"geom":rectangle("80.7260631215,12.5546640031 85.2928232587,17.4875933461")}
+{"id":97,"geom":rectangle("-44.180993763,-16.0807020012 -39.6192096547,-13.9933902095")}
+{"id":98,"geom":rectangle("26.0438041781,-5.57090292895 27.0334370304,-2.87316728612")}
+{"id":99,"geom":rectangle("81.8095539196,11.9399712319 83.7438057534,13.072433318")}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp
new file mode 100644
index 0000000..0cbadc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_dynamic_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset 
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset 
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE spatial_intersect(ps.geom, ls.geom);
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp
new file mode 100644
index 0000000..4dbf727
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/spatial_intersect_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset 
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset 
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp
new file mode 100644
index 0000000..09da36d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_distance_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: Geometry
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: Geometry
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ st_distance(ps.geom, ls.geom) < 1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp
new file mode 100644
index 0000000..b93e4ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/spatial_join/st_intersects_static_partitioning.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: Geometry
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: Geometry
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Run query with spatial partitioning hint
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ st_intersects(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index da74507..0f996d1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -172,8 +172,8 @@
                                                       -- ASSIGN  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$230, $$232][$$224, $$225]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$230, $$232]  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$233, $$235][$$224, $$225]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$233, $$235]  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ASSIGN  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
index 35fdc41..ece21bd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_0.plan
@@ -1,25 +1,53 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- NESTED_LOOP  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
-                      -- STREAM_PROJECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ASSIGN  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (test.OfficerLocations.OfficerLocations)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- DATASOURCE_SCAN (test.OfficerLocations)  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- SPLIT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- STREAM_SELECT  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SPLIT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- RTREE_SEARCH (test.OfficerLocations.o_location)  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
index 35fdc41..aa7008a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter_on_meta_with_idx_1.plan
@@ -6,9 +6,9 @@
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- NESTED_LOOP  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- STREAM_SELECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- DATASOURCE_SCAN (test.KVStore)  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
new file mode 100644
index 0000000..ce4f736
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_dynamic_partitioning.plan
@@ -0,0 +1,156 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- NESTED_LOOP  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- SPATIAL_JOIN [$$60, $$49] [$$61, $$50]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$60(ASC), $$49(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$60]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- UNNEST  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                      -- NESTED_LOOP  |UNPARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STABLE_SORT [$$61(ASC), $$50(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- UNNEST  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- NESTED_LOOP  |PARTITIONED|
+                                    -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                      -- NESTED_LOOP  |UNPARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                                          -- AGGREGATE  |UNPARTITIONED|
+                                                            -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                                              -- AGGREGATE  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- REPLICATE  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- NESTED_LOOP  |UNPARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                  -- AGGREGATE  |UNPARTITIONED|
+                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                      -- AGGREGATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                                  -- AGGREGATE  |UNPARTITIONED|
+                                    -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+                                      -- AGGREGATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
new file mode 100644
index 0000000..ce2f6e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/spatial_intersect_static_partitioning.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SPATIAL_JOIN [$$53, $$49] [$$54, $$50]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$53(ASC), $$49(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$54(ASC), $$50(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
new file mode 100644
index 0000000..25546ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_distance_static_partitioning.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SPATIAL_JOIN [$$61, $$56] [$$62, $$57]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$61(ASC), $$56(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$61]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$62(ASC), $$57(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$62]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
new file mode 100644
index 0000000..43d8da6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/spatial_join/st_intersects_static_partitioning.plan
@@ -0,0 +1,34 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- SPATIAL_JOIN [$$55, $$52] [$$56, $$53]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$55(ASC), $$52(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.ParkSet)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$56(ASC), $$53(ASC)]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$56]  |PARTITIONED|
+                          -- UNNEST  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.LakeSet)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
index a1173de..342c74d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/function/issue-2394/issue-2394.1.ddl.sqlpp
@@ -51,6 +51,7 @@
 let temp = (select * from Tweets f where f.user.id_str in (select value fw.followers from Followers fw where fw.twitter_id=user_id)[0]
 and
 spatial_intersect(create_point(f.place.bounding_box.coordinates[0][0][0],f.place.bounding_box.coordinates[0][0][1]),
-create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1])))
+create_point(t.place.bounding_box.coordinates[0][0][0],t.place.bounding_box.coordinates[0][0][1]))
+order by f.id)
 where t.user.id_str = user_id)
 };
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..4c521a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/hints_spatial_partitioning/hints_spatial_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..f123788
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp
new file mode 100644
index 0000000..aefab51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp
new file mode 100644
index 0000000..a56ea7f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp
new file mode 100644
index 0000000..4c521a5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp
new file mode 100644
index 0000000..5550d51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.ddl.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+-- Make Park type
+CREATE TYPE ParkType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Lake type
+CREATE TYPE LakeType as closed {
+    id: int32,
+    geom: rectangle
+};
+
+-- Make Park dataset
+CREATE DATASET ParkSet (ParkType) primary key id;
+
+-- Make Lake dataset
+CREATE DATASET LakeSet (LakeType) primary key id;
+
+-- Type for the geometry datasets
+CREATE TYPE GeomType AS{
+  id : int,
+  geom : geometry
+};
+
+-- Create datasets
+CREATE DATASET ParkSetG (GeomType) primary key id;
+CREATE DATASET LakeSetG (GeomType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp
new file mode 100644
index 0000000..2834e20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.2.update.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+LOAD DATASET ParkSet USING localfs((`path`=`asterix_nc1://data/spatial/parks.json`),(`format`=`adm`));
+LOAD DATASET LakeSet USING localfs((`path`=`asterix_nc1://data/spatial/lakes.json`),(`format`=`adm`));
+-- make geometries and insert into new datasets
+insert into LakeSetG select id, st_make_envelope(get_x (get_points (geom)[0]) ,get_y (get_points (geom)[0]) ,get_x (get_points (geom)[1]) ,get_y (get_points (geom)[1]) , 4326) geom from LakeSet;
+insert into ParkSetG select id, st_make_envelope(get_x (get_points (geom)[0]) ,get_y (get_points (geom)[0]) ,get_x (get_points (geom)[1]) ,get_y (get_points (geom)[1]) , 4326) geom from ParkSet;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp
new file mode 100644
index 0000000..2b59fc5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ParkSetG AS ps LEFT OUTER JOIN LakeSetG AS ls
+ON st_intersects(ps.geom, ls.geom)
+SELECT COUNT(*);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/hints_spatial_partitioning/hints_spatial_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_dynamic_partitioning/spatial_join_dynamic_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm
new file mode 100644
index 0000000..255c5f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_join_static_partitioning/spatial_join_static_partitioning.1.adm
@@ -0,0 +1 @@
+{ "$1": 34 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm
new file mode 100644
index 0000000..190a8f4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/spatial/spatial_left_outer_join_st_intersects/spatial_left_outer_join_st_intersects.1.adm
@@ -0,0 +1 @@
+{ "$1": 104 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 6d91771..65a676c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10005,6 +10005,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="spatial">
+      <compilation-unit name="hints_spatial_partitioning">
+        <output-dir compare="Text">hints_spatial_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
       <compilation-unit name="line_accessor">
         <output-dir compare="Text">line_accessor</output-dir>
       </compilation-unit>
@@ -10104,6 +10109,21 @@
         <output-dir compare="Text">spatial-large-data</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_join_dynamic_partitioning">
+        <output-dir compare="Text">spatial_join_dynamic_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_join_static_partitioning">
+        <output-dir compare="Text">spatial_join_static_partitioning</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="spatial">
+      <compilation-unit name="spatial_left_outer_join_st_intersects">
+        <output-dir compare="Text">spatial_left_outer_join_st_intersects</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="statement-params">
     <test-case FilePath="statement-params">
@@ -15147,7 +15167,7 @@
         <expected-warn>ASX1107: Unexpected hint: indexnl. "hash" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: hash. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: auto. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: hash. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: hash. None expected at this location</expected-warn>
       </compilation-unit>
     </test-case>
@@ -15170,7 +15190,7 @@
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_groupby. "hash" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_between. "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
-        <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "use-index" expected at this location</expected-warn>
+        <expected-warn>ASX1107: Unexpected hint: unknown_hint_funcall. "indexnl", "range", "skip-index", "spatial-partitioning", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_elsewhere. None expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
         <expected-warn>ASX1107: Unexpected hint: unknown_hint_relexpr_6. "hash-bcast", "indexnl", "skip-index", "use-index" expected at this location</expected-warn>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java
new file mode 100644
index 0000000..42262d6
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/annotations/SpatialJoinAnnotation.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.annotations;
+
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+
+public final class SpatialJoinAnnotation implements IExpressionAnnotation {
+
+    public static final String HINT_STRING = "spatial-partitioning";
+
+    private final double minX;
+    private final double minY;
+    private final double maxX;
+    private final double maxY;
+    private final int numRows;
+    private final int numColumns;
+
+    public SpatialJoinAnnotation(double minX, double minY, double maxX, double maxY, int numRows, int numColumns) {
+        this.minX = minX;
+        this.minY = minY;
+        this.maxX = maxX;
+        this.maxY = maxY;
+        this.numRows = numRows;
+        this.numColumns = numColumns;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("%s:%f,%f,%f,%f,%d,%d", HINT_STRING, getMinX(), getMinY(), getMaxX(), getMaxY(),
+                getNumRows(), getNumColumns());
+    }
+
+    public double getMinX() {
+        return minX;
+    }
+
+    public double getMinY() {
+        return minY;
+    }
+
+    public double getMaxX() {
+        return maxX;
+    }
+
+    public double getMaxY() {
+        return maxY;
+    }
+
+    public int getNumRows() {
+        return numRows;
+    }
+
+    public int getNumColumns() {
+        return numColumns;
+    }
+}
diff --git a/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md
new file mode 100644
index 0000000..ea89ef2
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join.md
@@ -0,0 +1,65 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+## <a id="spatial_joins">Spatial Joins</a>
+AsterixDB supports efficient spatial join query performance.
+In particular, it will execute the [Partition Based Spatial-Merge Join](http://pages.cs.wisc.edu/~dewitt/includes/paradise/spjoin.pdf).
+(PBSM) algorithm for the join queries with a spatial function in join condition.
+
+Supported spatial functions:
+- `spatial_intersect(ARectangle, ARectangle)`.
+- ESRI's spatial functions: `st_intersects()`, `st_overlaps()`, `st_touches()`, `st_contains()`, `st_crosses()`, `st_within()`, `st_distance()`.
+
+Once the join condition contains a supported spatial function, users do not need to do any further action to trigger this efficient query plan.
+
+### <a id="spatial_partitioning_hint">Using a spatial partitioning hint</a>
+PBSM algorithm requires the following information to partition data into a grid:
+- The MBR of two input datasets.
+- Number of rows and columns of the grid.
+
+By default, the MBR will be computed at running time and the grid size is 100x100.
+However, users can also set other values for these parameters using spatial partitioning hint.
+
+##### Spatial partitioning hint example
+In this example, assume that MBR of two input datasets is (-180.0, -83.0, 180.0, 90.0) and grid size is 10x10.
+
+
+    /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */
+
+
+##### Spatial partitioning hint in a query
+
+    DROP DATAVERSE test IF EXISTS;
+    CREATE DATAVERSE test;
+    USE test;
+
+    -- Make GeomType
+    CREATE TYPE GeomType as closed {
+        id: int32,
+        geom: rectangle
+    };
+
+    -- Make Park dataset
+    CREATE DATASET ParkSet (GeomType) primary key id;
+
+    -- Make Lake dataset
+    CREATE DATASET LakeSet (GeomType) primary key id;
+
+    SELECT COUNT(*) FROM ParkSet AS ps, LakeSet AS ls
+    WHERE /*+ spatial-partitioning(-180.0, -83.0, 180.0, 90.0, 10, 10) */ spatial_intersect(ps.geom, ls.geom);
diff --git a/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md
new file mode 100644
index 0000000..2edac18
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/spatial_join/spatial_join_title.md
@@ -0,0 +1,24 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+# Spatial Joins
+
+## <a id="#toc">Table of Contents</a> ##
+* [Introduction](#spatial_joins)
+* [Spatial partitioning hint](#spatial_partitioning_hint)
diff --git a/asterixdb/asterix-doc/src/site/site.xml b/asterixdb/asterix-doc/src/site/site.xml
index 8ce2c2a..5c837d2 100644
--- a/asterixdb/asterix-doc/src/site/site.xml
+++ b/asterixdb/asterix-doc/src/site/site.xml
@@ -99,6 +99,7 @@
       <item name="GIS Support Overview" href="geo/quickstart.html"/>
       <item name="GIS Functions" href="geo/functions.html"/>
       <item name="Support of Interval Joins" href="interval_join.html"/>
+      <item name="Support of Spatial Joins" href="spatial_join.html"/>
       <item name="Support of Array Indexes" href="sqlpp/arrayindex.html"/>
     </menu>
 
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
index deef79a..0e4d748 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/GeoFunctionRegistrant.java
@@ -53,6 +53,8 @@
 import org.apache.asterix.geo.evaluators.functions.STIsSimpleDescriptor;
 import org.apache.asterix.geo.evaluators.functions.STLengthDescriptor;
 import org.apache.asterix.geo.evaluators.functions.STLineFromMultiPointDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMBRDescriptor;
+import org.apache.asterix.geo.evaluators.functions.STMBREnlargeDescriptor;
 import org.apache.asterix.geo.evaluators.functions.STMDescriptor;
 import org.apache.asterix.geo.evaluators.functions.STMakeEnvelopeDescriptorSRID;
 import org.apache.asterix.geo.evaluators.functions.STMakePoint3DDescriptor;
@@ -153,5 +155,7 @@
         fc.add(STIntersectionDescriptor.FACTORY);
         fc.add(STSymDifferenceDescriptor.FACTORY);
         fc.add(STPolygonizeDescriptor.FACTORY);
+        fc.add(STMBRDescriptor.FACTORY);
+        fc.add(STMBREnlargeDescriptor.FACTORY);
     }
 }
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
index 29c2fd8..e41aebb 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/AbstractSTSingleGeometryDescriptor.java
@@ -29,6 +29,7 @@
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AGeometry;
 import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.ARectangle;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.EnumDeserializer;
@@ -122,6 +123,9 @@
                 } else if (finalResult instanceof OGCGeometry) {
                     out.writeByte(ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
                     AGeometrySerializerDeserializer.INSTANCE.serialize(new AGeometry((OGCGeometry) finalResult), out);
+                } else if (finalResult instanceof ARectangle) {
+                    SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE)
+                            .serialize(finalResult, out);
                 }
             } catch (IOException e) {
                 throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
new file mode 100644
index 0000000..55e358c
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBRDescriptor.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class STMBRDescriptor extends AbstractSTSingleGeometryDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMBRDescriptor();
+        }
+    };
+
+    @Override
+    protected Object evaluateOGCGeometry(OGCGeometry geometry) throws HyracksDataException {
+
+        AMutableRectangle aRectangle = new AMutableRectangle(null, null);
+        AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
+        Envelope env = new Envelope();
+        geometry.getEsriGeometry().queryEnvelope(env);
+        aPoint[0].setValue(env.getXMin(), env.getYMin());
+        aPoint[1].setValue(env.getXMax(), env.getYMax());
+        aRectangle.setValue(aPoint[0], aPoint[1]);
+        return aRectangle;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MBR;
+    }
+
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
new file mode 100644
index 0000000..b3bfb33
--- /dev/null
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMBREnlargeDescriptor.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.geo.evaluators.functions;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AGeometrySerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+import com.esri.core.geometry.Envelope;
+import com.esri.core.geometry.ogc.OGCGeometry;
+
+public class STMBREnlargeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new STMBREnlargeDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.ST_MBR_ENLARGE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private DataOutput out = resultStorage.getDataOutput();
+                    private IPointable inputArg0 = new VoidPointable();
+                    private IPointable inputArg1 = new VoidPointable();
+                    private IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+                    private ByteArrayAccessibleInputStream inStream =
+                            new ByteArrayAccessibleInputStream(new byte[0], 0, 0);
+                    private DataInputStream dataIn = new DataInputStream(inStream);
+
+                    Envelope env = new Envelope();
+
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer<ARectangle> rectangleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        resultStorage.reset();
+
+                        eval0.evaluate(tuple, inputArg0);
+                        eval1.evaluate(tuple, inputArg1);
+
+                        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
+                            return;
+                        }
+
+                        byte[] data0 = inputArg0.getByteArray();
+                        int offset0 = inputArg0.getStartOffset();
+                        int len = inputArg0.getLength();
+
+                        byte[] data1 = inputArg1.getByteArray();
+                        int offset1 = inputArg1.getStartOffset();
+
+                        if (data0[offset0] != ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, data0[offset0],
+                                    ATypeTag.SERIALIZED_GEOMETRY_TYPE_TAG);
+                        }
+
+                        inStream.setContent(data0, offset0 + 1, len - 1);
+                        OGCGeometry geometry =
+                                AGeometrySerializerDeserializer.INSTANCE.deserialize(dataIn).getGeometry();
+                        geometry.getEsriGeometry().queryEnvelope(env);
+                        double expandValue =
+                                ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, data1, offset1);
+                        AMutableRectangle expandedMBR = new AMutableRectangle(
+                                new AMutablePoint(env.getXMin() - expandValue, env.getYMin() - expandValue),
+                                new AMutablePoint(env.getXMax() + expandValue, env.getYMax() + expandValue));
+                        rectangleSerde.serialize(expandedMBR, out);
+                        result.set(resultStorage);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
index 8615268..191e6c3 100644
--- a/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
+++ b/asterixdb/asterix-geo/src/main/java/org/apache/asterix/geo/evaluators/functions/STMakeEnvelopeDescriptorSRID.java
@@ -103,6 +103,9 @@
 
         @Override
         public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+
+            resultStorage.reset();
+
             eval0.evaluate(tuple, inputArg0);
             byte[] data0 = inputArg0.getByteArray();
             int offset0 = inputArg0.getStartOffset();
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
index e6a3deb..cab1aca 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/ExpressionUtils.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.lang.common.util;
 
+import static org.apache.asterix.lang.common.base.Literal.Type.DOUBLE;
+
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -41,6 +43,8 @@
 import org.apache.asterix.lang.common.expression.LiteralExpr;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.literal.DoubleLiteral;
+import org.apache.asterix.lang.common.literal.FloatLiteral;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
 import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
@@ -55,6 +59,8 @@
 import org.apache.asterix.object.base.AdmObjectNode;
 import org.apache.asterix.object.base.AdmStringNode;
 import org.apache.asterix.object.base.IAdmNode;
+import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.types.ATypeTag;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
@@ -147,6 +153,74 @@
         return null;
     }
 
+    public static Literal reverseSign(Literal lit) throws TypeMismatchException {
+        switch (lit.getLiteralType()) {
+            case DOUBLE:
+                DoubleLiteral dLit = (DoubleLiteral) lit;
+                DoubleLiteral reversedDLit = new DoubleLiteral(-dLit.getValue());
+                return reversedDLit;
+            case FLOAT:
+                FloatLiteral fLit = (FloatLiteral) lit;
+                FloatLiteral reversedFLit = new FloatLiteral(-fLit.getValue());
+                return reversedFLit;
+            case LONG:
+                LongIntegerLiteral lLit = (LongIntegerLiteral) lit;
+                LongIntegerLiteral reversedLLit = new LongIntegerLiteral(-lLit.getValue());
+                return reversedLLit;
+            case INTEGER:
+                IntegerLiteral iLit = (IntegerLiteral) lit;
+                IntegerLiteral reversedILit = new IntegerLiteral(-iLit.getValue());
+                return reversedILit;
+            case NULL:
+            case MISSING:
+                return lit;
+            default:
+                throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(lit), ATypeTag.DOUBLE,
+                        ATypeTag.FLOAT, ATypeTag.BIGINT, ATypeTag.INTEGER);
+        }
+    }
+
+    public static double getDoubleValue(Literal item) throws TypeMismatchException {
+        if ((item.getLiteralType() == Literal.Type.DOUBLE) || (item.getLiteralType() == Literal.Type.FLOAT)
+                || (item.getLiteralType() == Literal.Type.LONG) || (item.getLiteralType() == Literal.Type.INTEGER)) {
+            return ((Number) item.getValue()).doubleValue();
+        } else {
+            throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(item), ATypeTag.DOUBLE,
+                    ATypeTag.FLOAT, ATypeTag.BIGINT, ATypeTag.INTEGER);
+        }
+    }
+
+    public static long getLongValue(Literal item) throws TypeMismatchException {
+        if ((item.getLiteralType() == Literal.Type.LONG) || (item.getLiteralType() == Literal.Type.INTEGER)) {
+            return ((Number) item.getValue()).longValue();
+        } else {
+            throw new TypeMismatchException(null, convertLiteralTypeTagToATypeTag(item), ATypeTag.BIGINT,
+                    ATypeTag.INTEGER);
+        }
+    }
+
+    private static ATypeTag convertLiteralTypeTagToATypeTag(Literal lit) {
+        switch (lit.getLiteralType()) {
+            case DOUBLE:
+                return ATypeTag.DOUBLE;
+            case FLOAT:
+                return ATypeTag.FLOAT;
+            case LONG:
+                return ATypeTag.BIGINT;
+            case INTEGER:
+                return ATypeTag.INTEGER;
+            case TRUE:
+            case FALSE:
+                return ATypeTag.BOOLEAN;
+            case NULL:
+                return ATypeTag.NULL;
+            case MISSING:
+                return ATypeTag.MISSING;
+            default:
+                return ATypeTag.STRING;
+        }
+    }
+
     public static void collectDependencies(Expression expression, IQueryRewriter rewriter,
             List<Triple<DataverseName, String, String>> outDatasetDependencies,
             List<Triple<DataverseName, String, String>> outSynonymDependencies,
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
index ea199ea..925140b 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/parser/SqlppHint.java
@@ -41,6 +41,7 @@
     LIST_VAL_FILE_HINT("list-val-file"),
     RANGE_HINT("range"),
     SKIP_SECONDARY_INDEX_SEARCH_HINT("skip-index"),
+    SPATIAL_JOIN_HINT("spatial-partitioning"),
     USE_SECONDARY_INDEX_SEARCH_HINT("use-index"),
     VAL_FILE_HINT("val-files"),
     VAL_FILE_SAME_INDEX_HINT("val-file-same-idx"),
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 502d3ae..969d579 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -63,6 +63,7 @@
 import org.apache.asterix.common.annotations.RangeAnnotation;
 import org.apache.asterix.common.annotations.SecondaryIndexSearchPreferenceAnnotation;
 import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
+import org.apache.asterix.common.annotations.SpatialJoinAnnotation;
 import org.apache.asterix.common.annotations.TypeDataGen;
 import org.apache.asterix.common.annotations.UndeclaredFieldsDataGen;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
@@ -171,6 +172,7 @@
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.ExpressionUtils;
 import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
@@ -203,6 +205,7 @@
 import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
+import org.apache.asterix.om.exceptions.TypeMismatchException;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.commons.lang3.ArrayUtils;
@@ -401,6 +404,19 @@
         return new SQLPPParser(text).parseParenthesizedIdentifierList();
     }
 
+    private List<Literal> parseParenthesizedLiteralList() throws CompilationException {
+        return parseImpl(new ParseFunction<List<Literal>>() {
+            @Override
+            public  List<Literal> parse() throws ParseException {
+                return SQLPPParser.this.ParenthesizedLiteralList();
+            }
+        });
+    }
+
+    private static List<Literal> parseParenthesizedLiteralList(String text) throws CompilationException {
+        return new SQLPPParser(text).parseParenthesizedLiteralList();
+    }
+
     @Override
     public FunctionDecl parseFunctionBody(FunctionSignature signature, List<String> paramNames, boolean isStored)
       throws CompilationException {
@@ -655,6 +671,47 @@
               List<String> indexNames = parseParenthesizedIdentifierList(hintToken.hintParams);
               return SkipSecondaryIndexSearchExpressionAnnotation.newInstance(indexNames);
             }
+          case SPATIAL_JOIN_HINT:
+            List<Literal> hintValues = parseParenthesizedLiteralList(hintToken.hintParams);
+
+            // Handle exceptions
+            if (hintValues.size() != 6) {
+              throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. 6 arguments are required.",
+                                hintToken.hint.toString()));
+            }
+
+            for (int i = 0; i < 4; i++) {
+              Literal lit = hintValues.get(i);
+              if ((lit.getLiteralType() != Literal.Type.DOUBLE)
+                    && (lit.getLiteralType() != Literal.Type.FLOAT)
+                    && (lit.getLiteralType() != Literal.Type.LONG)
+                    && (lit.getLiteralType() != Literal.Type.INTEGER)){
+                throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. Numeric value is required for first 4 arguments.",
+                                hintToken.hint.toString()));
+              }
+            }
+
+            for (int i = 4; i < 6; i++) {
+              Literal lit = hintValues.get(i);
+              if ((lit.getLiteralType() != Literal.Type.LONG)
+                    && (lit.getLiteralType() != Literal.Type.INTEGER)) {
+                throw new SqlppParseException(getSourceLocation(hintToken), String.format("Unexpected hint: %s. Long/int is required for last 2 arguments.",
+                              hintToken.hint.toString()));
+              }
+            }
+
+            try {
+              double minX = ExpressionUtils.getDoubleValue(hintValues.get(0));
+              double minY = ExpressionUtils.getDoubleValue(hintValues.get(1));
+              double maxX = ExpressionUtils.getDoubleValue(hintValues.get(2));
+              double maxY = ExpressionUtils.getDoubleValue(hintValues.get(3));
+              int numRows = (int) ExpressionUtils.getLongValue(hintValues.get(4));
+              int numColumns = (int) ExpressionUtils.getLongValue(hintValues.get(5));
+              SpatialJoinAnnotation spatialJoinAnn = new SpatialJoinAnnotation(minX, minY, maxX, maxY, numRows, numColumns);
+              return spatialJoinAnn;
+            } catch (TypeMismatchException e) {
+              throw new SqlppParseException(getSourceLocation(hintToken), e.getMessage());
+            }
           case USE_SECONDARY_INDEX_SEARCH_HINT:
             if (hintToken.hintParams == null) {
               throw new SqlppParseException(getSourceLocation(hintToken), "Expected index name(s)");
@@ -2900,8 +2957,8 @@
   // Note: there's a copy of this production in PrimaryExpr() (LOOKAHEAD for FunctionCallExpr())
   //       that copy must be kept in sync with this code
   prefix = MultipartIdentifierWithHints(
-    SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT,
-    SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
+    SqlppHint.INDEXED_NESTED_LOOP_JOIN_HINT, SqlppHint.RANGE_HINT, SqlppHint.SKIP_SECONDARY_INDEX_SEARCH_HINT,
+    SqlppHint.SPATIAL_JOIN_HINT, SqlppHint.USE_SECONDARY_INDEX_SEARCH_HINT
   )
   (<SHARP> suffix = Identifier())?
   {
@@ -2980,6 +3037,49 @@
   }
 }
 
+List<Literal> ParenthesizedLiteralList() throws ParseException:
+{
+  List<Literal> list = new ArrayList<Literal>();
+  boolean minus = false;
+  Expression litExpr = null;
+  Literal lit = null;
+}
+{
+  <LEFTPAREN>
+  (<MINUS> { minus = true; })? litExpr = Literal()
+  {
+    lit = ((LiteralExpr) litExpr).getValue();
+    if (minus)
+    {
+      try {
+        lit = ExpressionUtils.reverseSign(lit);
+      } catch (TypeMismatchException e) {
+        throw new SqlppParseException(getSourceLocation(token), e.getMessage());
+      }
+      minus = false;
+    }
+    list.add(lit);
+  }
+  ( <COMMA> (<MINUS> { minus = true; })? litExpr = Literal()
+  {
+    lit = ((LiteralExpr) litExpr).getValue();
+    if (minus)
+    {
+      try {
+        lit = ExpressionUtils.reverseSign(lit);
+      } catch (TypeMismatchException e) {
+        throw new SqlppParseException(getSourceLocation(token), e.getMessage());
+      }
+      minus = false;
+    }
+    list.add(lit);
+  } )*
+  <RIGHTPAREN>
+  {
+    return list;
+  }
+}
+
 Pair<Integer, Pair<List<String>, IndexedTypeExpression>> OpenField() throws ParseException:
 {
   IndexedTypeExpression fieldType = null;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 3ce4f81..2009e0f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -134,6 +134,7 @@
 import org.apache.asterix.om.typecomputer.impl.TreatAsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryBinaryInt64TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UniformInputTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.UnionMbrAggTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnorderedListConstructorTypeComputer;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -607,6 +608,14 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-kurtosis", 1);
     public static final FunctionIdentifier NULL_WRITER =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-null-writer", 1);
+    public static final FunctionIdentifier UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-union_mbr", 1);
+    public static final FunctionIdentifier LOCAL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-union_mbr", 1);
+    public static final FunctionIdentifier INTERMEDIATE_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-union_mbr", 1);
+    public static final FunctionIdentifier GLOBAL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-union_mbr", 1);
 
     public static final FunctionIdentifier SCALAR_ARRAYAGG =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "arrayagg", 1);
@@ -634,6 +643,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "skewness", 1);
     public static final FunctionIdentifier SCALAR_KURTOSIS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "kurtosis", 1);
+    public static final FunctionIdentifier SCALAR_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "union_mbr", 1);
 
     // serializable aggregate functions
     public static final FunctionIdentifier SERIAL_AVG =
@@ -836,6 +847,14 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-kurtosis", 1);
     public static final FunctionIdentifier LOCAL_SQL_KURTOSIS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-kurtosis", 1);
+    public static final FunctionIdentifier SQL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sql-union_mbr", 1);
+    public static final FunctionIdentifier LOCAL_SQL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sql-union_mbr", 1);
+    public static final FunctionIdentifier INTERMEDIATE_SQL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-sql-union_mbr", 1);
+    public static final FunctionIdentifier GLOBAL_SQL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-global-sql-union_mbr", 1);
 
     public static final FunctionIdentifier SCALAR_SQL_AVG =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-avg", 1);
@@ -859,6 +878,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-skewness", 1);
     public static final FunctionIdentifier SCALAR_SQL_KURTOSIS =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-kurtosis", 1);
+    public static final FunctionIdentifier SCALAR_SQL_UNION_MBR =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sql-union_mbr", 1);
 
     // serializable sql aggregate functions
     public static final FunctionIdentifier SERIAL_SQL_AVG =
@@ -1033,6 +1054,8 @@
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "subset-collection", 3);
 
     public static final FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
+    public static final FunctionIdentifier SPATIAL_TILE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "spatial-tile", 4);
 
     // fuzzy functions
     public static final FunctionIdentifier FUZZY_EQ =
@@ -1247,6 +1270,10 @@
     public static final FunctionIdentifier CAST_TYPE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cast", 1);
     public static final FunctionIdentifier CAST_TYPE_LAX =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "cast-lax", 1);
+    public static final FunctionIdentifier REFERENCE_TILE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reference-tile", 6);
+    public static final FunctionIdentifier GET_INTERSECTION =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-intersection", 2);
 
     public static final FunctionIdentifier CREATE_UUID =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "create-uuid", 0);
@@ -1390,6 +1417,10 @@
     public static final FunctionIdentifier ST_POLYGONIZE =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-polygonize", 1);
 
+    public static final FunctionIdentifier ST_MBR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-mbr", 1);
+    public static final FunctionIdentifier ST_MBR_ENLARGE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "st-mbr-enlarge", 2);
+
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-year", 1);
@@ -1869,6 +1900,8 @@
                 new ScalarVersionOfAggregateResultType(NumericSumAggTypeComputer.INSTANCE);
         ScalarVersionOfAggregateResultType scalarMinMaxTypeComputer =
                 new ScalarVersionOfAggregateResultType(MinMaxAggTypeComputer.INSTANCE);
+        ScalarVersionOfAggregateResultType scalarUnionMbrTypeComputer =
+                new ScalarVersionOfAggregateResultType(UnionMbrAggTypeComputer.INSTANCE);
 
         addPrivateFunction(LISTIFY, OrderedListConstructorTypeComputer.INSTANCE, true);
         addFunction(SCALAR_ARRAYAGG, ScalarArrayAggTypeComputer.INSTANCE, true);
@@ -1912,6 +1945,10 @@
         addFunction(KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(NULL_WRITER, PropagateTypeComputer.INSTANCE_NULLABLE, true);
+        addFunction(UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(LOCAL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERMEDIATE_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(GLOBAL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
 
         // SUM
         addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1970,6 +2007,7 @@
         addPrivateFunction(SERIAL_GLOBAL_SQL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_LOCAL_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_INTERMEDIATE_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_UNION_MBR, scalarUnionMbrTypeComputer, true);
 
         // SQL SUM
         addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -2030,6 +2068,11 @@
         addPrivateFunction(LOCAL_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addPrivateFunction(INTERMEDIATE_SQL_KURTOSIS, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addFunction(SCALAR_SQL_KURTOSIS, NullableDoubleTypeComputer.INSTANCE, true);
+        addFunction(SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(LOCAL_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(INTERMEDIATE_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(GLOBAL_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addFunction(SCALAR_SQL_UNION_MBR, ARectangleTypeComputer.INSTANCE, true);
 
         addPrivateFunction(SERIAL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_COUNT, AInt64TypeComputer.INSTANCE, true);
@@ -2174,6 +2217,9 @@
         addFunction(GET_CIRCLE_RADIUS_ACCESSOR, ADoubleTypeComputer.INSTANCE, true);
         addFunction(GET_CIRCLE_CENTER_ACCESSOR, APointTypeComputer.INSTANCE, true);
         addFunction(GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR, OrderedListOfAPointTypeComputer.INSTANCE, true);
+        addPrivateFunction(SPATIAL_TILE, AInt32TypeComputer.INSTANCE, true);
+        addPrivateFunction(REFERENCE_TILE, AInt32TypeComputer.INSTANCE, true);
+        addPrivateFunction(GET_INTERSECTION, ARectangleTypeComputer.INSTANCE, true);
 
         //geo functions
         addFunction(ST_AREA, ADoubleTypeComputer.INSTANCE, true);
@@ -2244,6 +2290,9 @@
         addPrivateFunction(ST_UNION_SQL_AGG, AGeometryTypeComputer.INSTANCE, true);
         addFunction(ST_POLYGONIZE, AGeometryTypeComputer.INSTANCE, true);
 
+        addPrivateFunction(ST_MBR, ARectangleTypeComputer.INSTANCE, true);
+        addPrivateFunction(ST_MBR_ENLARGE, ARectangleTypeComputer.INSTANCE, true);
+
         // Binary functions
         addFunction(BINARY_HEX_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(BINARY_BASE64_CONSTRUCTOR, ABinaryTypeComputer.INSTANCE_NULLABLE, true);
@@ -3066,6 +3115,18 @@
         addScalarAgg(ST_UNION_SQL_AGG, SCALAR_ST_UNION_SQL_AGG);
         addDistinctAgg(ST_UNION_SQL_AGG_DISTINCT, ST_UNION_SQL_AGG);
         addScalarAgg(ST_UNION_SQL_AGG_DISTINCT, SCALAR_ST_UNION_SQL_AGG_DISTINCT);
+
+        // SQL UNION MBR
+        addAgg(SQL_UNION_MBR);
+        addAgg(LOCAL_SQL_UNION_MBR);
+        addAgg(GLOBAL_SQL_UNION_MBR);
+        addLocalAgg(SQL_UNION_MBR, LOCAL_SQL_UNION_MBR);
+        addIntermediateAgg(LOCAL_SQL_UNION_MBR, INTERMEDIATE_SQL_UNION_MBR);
+        addIntermediateAgg(GLOBAL_SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+        addIntermediateAgg(SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+        addGlobalAgg(SQL_UNION_MBR, GLOBAL_SQL_UNION_MBR);
+
+        addScalarAgg(SQL_UNION_MBR, SCALAR_SQL_UNION_MBR);
     }
 
     interface BuiltinFunctionProperty {
@@ -3110,6 +3171,7 @@
         addUnnestFun(RANGE, true);
         addUnnestFun(SCAN_COLLECTION, false);
         addUnnestFun(SUBSET_COLLECTION, false);
+        addUnnestFun(SPATIAL_TILE, false);
     }
 
     public enum DataSourceFunctionProperty implements BuiltinFunctionProperty {
@@ -3306,11 +3368,18 @@
     }
 
     public enum SpatialFilterKind {
-        SI
+        SI,
+        STFR
     }
 
     static {
         spatialFilterFunctions.put(BuiltinFunctions.SPATIAL_INTERSECT, SpatialFilterKind.SI);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_INTERSECTS, SpatialFilterKind.STFR);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_OVERLAPS, SpatialFilterKind.STFR);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_TOUCHES, SpatialFilterKind.STFR);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_CONTAINS, SpatialFilterKind.STFR);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_CROSSES, SpatialFilterKind.STFR);
+        spatialFilterFunctions.put(BuiltinFunctions.ST_WITHIN, SpatialFilterKind.STFR);
     }
 
     public static boolean isGlobalAggregateFunction(FunctionIdentifier fi) {
@@ -3318,7 +3387,11 @@
     }
 
     public static boolean isSpatialFilterFunction(FunctionIdentifier fi) {
-        return spatialFilterFunctions.get(fi) != null;
+        return spatialFilterFunctions.get(fi) == SpatialFilterKind.SI;
+    }
+
+    public static boolean isSTFilterRefineFunction(FunctionIdentifier fi) {
+        return spatialFilterFunctions.get(fi) == SpatialFilterKind.STFR;
     }
 
     static {
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java
new file mode 100644
index 0000000..ed03b21
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/UnionMbrAggTypeComputer.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class UnionMbrAggTypeComputer extends AggregateResultTypeComputer {
+
+    public static final UnionMbrAggTypeComputer INSTANCE = new UnionMbrAggTypeComputer();
+
+    private UnionMbrAggTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        ATypeTag tag = strippedInputTypes[0].getTypeTag();
+        switch (tag) {
+            case RECTANGLE:
+                return AUnionType.createNullableType(BuiltinType.ARECTANGLE);
+            case ANY:
+                return BuiltinType.ANY;
+            default:
+                // All other possible cases.
+                return BuiltinType.ANULL;
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..655bc8a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.std.SqlUnionMbrAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarSqlUnionMbrAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            createDescriptorFactory(ScalarSqlUnionMbrAggregateDescriptor::new);
+
+    private ScalarSqlUnionMbrAggregateDescriptor() {
+        super(SqlUnionMbrAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SCALAR_SQL_UNION_MBR;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        super.setImmutableStates(states);
+        aggFuncDesc.setImmutableStates(getItemType((IAType) states[0]));
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..188508c
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/scalar/ScalarUnionMbrAggregateDescriptor.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.scalar;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.std.UnionMbrAggregateDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+
+public class ScalarUnionMbrAggregateDescriptor extends AbstractScalarAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY =
+            createDescriptorFactory(ScalarUnionMbrAggregateDescriptor::new);
+
+    private ScalarUnionMbrAggregateDescriptor() {
+        super(UnionMbrAggregateDescriptor.FACTORY);
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SCALAR_UNION_MBR;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        super.setImmutableStates(states);
+        aggFuncDesc.setImmutableStates(getItemType((IAType) states[0]));
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..4840f52
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateDescriptor.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+
+public abstract class AbstractUnionMbrAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    IAType aggFieldType;
+
+    @Override
+    public void setImmutableStates(Object... types) {
+        aggFieldType = (IAType) types[0];
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java
new file mode 100644
index 0000000..9414365
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractUnionMbrAggregateFunction.java
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.aggregates.std;
+
+import java.io.IOException;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public abstract class AbstractUnionMbrAggregateFunction extends AbstractAggregateFunction {
+
+    private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    private IPointable inputVal = new VoidPointable();
+    private final IScalarEvaluator eval;
+    protected final IEvaluatorContext context;
+    protected double currentMinX;
+    protected double currentMinY;
+    protected double currentMaxX;
+    protected double currentMaxY;
+
+    protected final AMutablePoint[] aPoint = { new AMutablePoint(0.0, 0.0), new AMutablePoint(0.0, 0.0) };
+    protected final AMutableRectangle aRect = new AMutableRectangle(aPoint[0], aPoint[1]);
+
+    private ISerializerDeserializer<ARectangle> rectangleSerde =
+            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+    public AbstractUnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(sourceLoc);
+        this.eval = args[0].createScalarEvaluator(context);
+        this.context = context;
+    }
+
+    @Override
+    public void init() throws HyracksDataException {
+        // Initialize the resulting mbr coordinates
+        currentMinX = Double.POSITIVE_INFINITY;
+        currentMinY = Double.POSITIVE_INFINITY;
+        currentMaxX = Double.NEGATIVE_INFINITY;
+        currentMaxY = Double.NEGATIVE_INFINITY;
+    }
+
+    @Override
+    public void step(IFrameTupleReference tuple) throws HyracksDataException {
+        eval.evaluate(tuple, inputVal);
+        byte[] data = inputVal.getByteArray();
+        int offset = inputVal.getStartOffset();
+        int len = inputVal.getLength();
+        ATypeTag typeTag =
+                EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[inputVal.getStartOffset()]);
+        if (typeTag == ATypeTag.NULL || typeTag == ATypeTag.MISSING) {
+            processNull(typeTag);
+        } else if (typeTag == ATypeTag.RECTANGLE) {
+            double minX = ADoubleSerializerDeserializer.getDouble(data,
+                    offset + 1 + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+            double minY = ADoubleSerializerDeserializer.getDouble(data,
+                    offset + 1 + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+            double maxX = ADoubleSerializerDeserializer.getDouble(data,
+                    offset + 1 + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+            double maxY = ADoubleSerializerDeserializer.getDouble(data,
+                    offset + 1 + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+            currentMinX = Math.min(currentMinX, minX);
+            currentMinY = Math.min(currentMinY, minY);
+            currentMaxX = Math.max(currentMaxX, maxX);
+            currentMaxY = Math.max(currentMaxY, maxY);
+        }
+    }
+
+    @Override
+    public void finish(IPointable result) throws HyracksDataException {
+        resultStorage.reset();
+        try {
+            aPoint[0].setValue(currentMinX, currentMinY);
+            aPoint[1].setValue(currentMaxX, currentMaxY);
+            aRect.setValue(aPoint[0], aPoint[1]);
+            rectangleSerde.serialize(aRect, resultStorage.getDataOutput());
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+        result.set(resultStorage);
+    }
+
+    @Override
+    public void finishPartial(IPointable result) throws HyracksDataException {
+        if (!isValidCoordinates(currentMinX, currentMinY, currentMaxX, currentMaxY)) {
+            currentMinX = 0.0;
+            currentMinY = 0.0;
+            currentMaxX = 0.0;
+            currentMaxY = 0.0;
+        }
+
+        finish(result);
+    }
+
+    protected void processNull(ATypeTag typeTag) throws UnsupportedItemTypeException {
+        throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.UNION_MBR, typeTag.serialize());
+    }
+
+    private boolean isValidCoordinates(double minX, double minY, double maxX, double maxY) {
+        return (minX != Double.POSITIVE_INFINITY) && (minY != Double.POSITIVE_INFINITY)
+                && (maxX != Double.NEGATIVE_INFINITY) && (maxY != Double.NEGATIVE_INFINITY);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..218ae39
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(GlobalSqlUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GLOBAL_SQL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..5b66bf4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/GlobalUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class GlobalUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(GlobalUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GLOBAL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..52a517b
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(IntermediateSqlUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.INTERMEDIATE_SQL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..0d66149
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/IntermediateUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntermediateUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(IntermediateUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.INTERMEDIATE_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..3ee1e47
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LocalSqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(LocalSqlUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.LOCAL_SQL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..0b5cbdd
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class LocalUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(LocalUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.LOCAL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..2dd125e
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SqlUnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY =
+            AbstractAggregateFunctionDynamicDescriptor.createFactory(SqlUnionMbrAggregateDescriptor::new);
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SQL_UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new SqlUnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java
new file mode 100644
index 0000000..c101a69
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/SqlUnionMbrAggregateFunction.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class SqlUnionMbrAggregateFunction extends AbstractUnionMbrAggregateFunction {
+    public SqlUnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java
new file mode 100644
index 0000000..33aa766
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateDescriptor.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IAggregateEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class UnionMbrAggregateDescriptor extends AbstractUnionMbrAggregateDescriptor {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = () -> new UnionMbrAggregateDescriptor();
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.UNION_MBR;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                return new UnionMbrAggregateFunction(args, ctx, sourceLoc);
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java
new file mode 100644
index 0000000..e0fc3e7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/UnionMbrAggregateFunction.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.aggregates.std;
+
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public class UnionMbrAggregateFunction extends AbstractUnionMbrAggregateFunction {
+
+    public UnionMbrAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
+            SourceLocation sourceLoc) throws HyracksDataException {
+        super(args, context, sourceLoc);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
index 7eb7358..192cf95 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/SpatialUtils.java
@@ -21,6 +21,7 @@
 import org.apache.asterix.dataflow.data.nontagged.Coordinate;
 import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import org.apache.asterix.om.base.ARectangle;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class SpatialUtils {
@@ -177,4 +178,18 @@
     public static void setTriangleYCoordinate(DoubleArray trianglesY, int triangleId, int point, double value) {
         trianglesY.get()[triangleId * 3 + point] = value;
     }
+
+    public static boolean intersects(ARectangle rect1, ARectangle rect2) {
+        // If one rectangle is on left side of other
+        if ((rect1.getP1().getX() > rect2.getP2().getX()) || (rect2.getP1().getX() > rect1.getP2().getX())) {
+            return false;
+        }
+
+        // If one rectangle is above other
+        if ((rect1.getP1().getY() > rect2.getP2().getY()) || (rect2.getP1().getY() > rect1.getP2().getY())) {
+            return false;
+        }
+
+        return true;
+    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java
new file mode 100644
index 0000000..e37fa05
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetIntersectionDescriptor.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutablePoint;
+import org.apache.asterix.om.base.AMutableRectangle;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class GetIntersectionDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = () -> new GetIntersectionDescriptor();
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.GET_INTERSECTION;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+                final IHyracksTaskContext hyracksTaskContext = ctx.getTaskContext();
+
+                return new IScalarEvaluator() {
+
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private final IPointable inputArg0 = new VoidPointable();
+                    private final IPointable inputArg1 = new VoidPointable();
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+
+                    private final AMutableRectangle aRectangle =
+                            new AMutableRectangle(new AMutablePoint(0.0, 0.0), new AMutablePoint(0.0, 0.0));
+                    private final AMutablePoint[] aPoint = { new AMutablePoint(0, 0), new AMutablePoint(0, 0) };
+
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer<ARectangle> rectangleSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ARECTANGLE);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        resultStorage.reset();
+
+                        eval0.evaluate(tuple, inputArg0);
+                        eval1.evaluate(tuple, inputArg1);
+
+                        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
+                            return;
+                        }
+
+                        byte[] bytes0 = inputArg0.getByteArray();
+                        byte[] bytes1 = inputArg1.getByteArray();
+
+                        int offset0 = inputArg0.getStartOffset();
+                        int offset1 = inputArg1.getStartOffset();
+
+                        ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                        ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+
+                        if (tag0 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+                        if (tag1 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 1, bytes1[offset1],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+
+                        double ax1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double ay1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double ax2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+                        double ay2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+                        double bx1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double by1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double bx2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+                        double by2 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+                        // Bottom-left of the intersection rectangle
+                        double ix1 = Math.max(ax1, bx1);
+                        double iy1 = Math.max(ay1, by1);
+
+                        // Top-right of the intersection rectangle
+                        double ix2 = Math.min(ax2, bx2);
+                        double iy2 = Math.min(ay2, by2);
+
+                        // Update the intersection rectangle.
+                        // If there is no intersection, return default rectangle [(0,0),(0,0)]
+                        if ((ix1 < ix2) && (iy1 < iy2)) {
+                            aPoint[0].setValue(ix1, iy1);
+                            aPoint[1].setValue(ix2, iy2);
+                        } else {
+                            aPoint[0].setValue(0.0, 0.0);
+                            aPoint[1].setValue(0.0, 0.0);
+                        }
+                        aRectangle.setValue(aPoint[0], aPoint[1]);
+                        resultStorage.reset();
+                        rectangleSerde.serialize(aRectangle, resultStorage.getDataOutput());
+                        result.set(resultStorage);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
index 01aa181..4a4c36c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/PointableHelper.java
@@ -170,43 +170,49 @@
     // 1 pointable check
     public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1)
             throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, null, pointable1, null, null, null);
+        return checkAndSetMissingOrNull(result, null, pointable1, null, null, null, null);
     }
 
     // 2 pointables check
     public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2)
             throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, null, null);
+        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, null, null, null);
     }
 
     // 3 pointables check
     public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
             IPointable pointable3) throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, null);
+        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, null, null);
     }
 
     // 4 pointables check
     public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
             IPointable pointable3, IPointable pointable4) throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4);
+        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4, null);
+    }
+
+    // 5 pointables check
+    public static boolean checkAndSetMissingOrNull(IPointable result, IPointable pointable1, IPointable pointable2,
+            IPointable pointable3, IPointable pointable4, IPointable pointable5) throws HyracksDataException {
+        return checkAndSetMissingOrNull(result, null, pointable1, pointable2, pointable3, pointable4, pointable5);
     }
 
     // 1 pointable check (check list members for missing values)
     public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1)
             throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, listAccessor, pointable1, null, null, null);
+        return checkAndSetMissingOrNull(result, listAccessor, pointable1, null, null, null, null);
     }
 
     // 2 pointables check (check list members for missing values)
     public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
             IPointable pointable2) throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, null, null);
+        return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, null, null, null);
     }
 
     // 3 pointables check (check list members for missing values)
     public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
             IPointable pointable2, IPointable pointable3) throws HyracksDataException {
-        return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, pointable3, null);
+        return checkAndSetMissingOrNull(result, listAccessor, pointable1, pointable2, pointable3, null, null);
     }
 
     /**
@@ -227,11 +233,13 @@
      * @param pointable2 the second pointable to be checked
      * @param pointable3 the third pointable to be checked
      * @param pointable4 the fourth pointable to be checked
+     * @param pointable5 the fourth pointable to be checked
      *
      * @return {@code true} if the pointable value is missing or null, {@code false} otherwise.
      */
     public static boolean checkAndSetMissingOrNull(IPointable result, ListAccessor listAccessor, IPointable pointable1,
-            IPointable pointable2, IPointable pointable3, IPointable pointable4) throws HyracksDataException {
+            IPointable pointable2, IPointable pointable3, IPointable pointable4, IPointable pointable5)
+            throws HyracksDataException {
 
         // this flag will keep an eye on whether a null value is encountered or not
         boolean isMeetNull = false;
@@ -278,6 +286,17 @@
             }
         }
 
+        if (pointable5 != null) {
+            switch (getPointableValueState(pointable5, listAccessor)) {
+                case MISSING:
+                    setMissing(result);
+                    return true;
+                case NULL:
+                    isMeetNull = true;
+                    break;
+            }
+        }
+
         // this is reached only if no missing is encountered in all the passed pointables
         if (isMeetNull) {
             setNull(result);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java
new file mode 100644
index 0000000..c55aa75
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/ReferenceTileDescriptor.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ReferenceTileDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ReferenceTileDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.REFERENCE_TILE;
+    }
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
+                return new IScalarEvaluator() {
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private final IPointable inputArg0 = new VoidPointable();
+                    private final IPointable inputArg1 = new VoidPointable();
+                    private final IPointable inputArg2 = new VoidPointable();
+                    private final IPointable inputArg3 = new VoidPointable();
+                    private final IPointable inputArg4 = new VoidPointable();
+                    private final IPointable inputArg5 = new VoidPointable();
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval2 = args[2].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval3 = args[3].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval4 = args[4].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval5 = args[5].createScalarEvaluator(ctx);
+
+                    private final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer intSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        eval0.evaluate(tuple, inputArg0);
+                        eval1.evaluate(tuple, inputArg1);
+                        eval2.evaluate(tuple, inputArg2);
+                        eval3.evaluate(tuple, inputArg3);
+                        eval4.evaluate(tuple, inputArg4);
+                        eval5.evaluate(tuple, inputArg5);
+
+                        if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1, inputArg2, inputArg3,
+                                inputArg4)) {
+                            return;
+                        }
+
+                        byte[] bytes0 = inputArg0.getByteArray();
+                        byte[] bytes1 = inputArg1.getByteArray();
+                        byte[] bytes2 = inputArg2.getByteArray();
+                        byte[] bytes3 = inputArg3.getByteArray();
+                        byte[] bytes4 = inputArg4.getByteArray();
+
+                        int offset0 = inputArg0.getStartOffset();
+                        int offset1 = inputArg1.getStartOffset();
+                        int offset2 = inputArg2.getStartOffset();
+                        int offset3 = inputArg3.getStartOffset();
+                        int offset4 = inputArg4.getStartOffset();
+
+                        ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                        ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+                        ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]);
+
+                        if (tag0 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+                        if (tag1 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+                        if (tag2 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes2[offset2],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+
+                        double ax1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double ay1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double bx1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double by1 = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double minX = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double minY = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double maxX = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+                        double maxY = ADoubleSerializerDeserializer.getDouble(bytes2, offset2 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+                        int rows = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes3, offset3);
+                        int columns = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes4, offset4);
+
+                        // Compute the reference point
+                        double x = Math.max(ax1, bx1);
+                        double y = Math.max(ay1, by1);
+
+                        // Compute the tile ID of the reference point
+                        int row = (int) Math.ceil((y - minY) * rows / (maxY - minY));
+                        int col = (int) Math.ceil((x - minX) * columns / (maxX - minX));
+
+                        row = Math.min(Math.max(1, row), rows * columns);
+                        col = Math.min(Math.max(1, col), rows * columns);
+
+                        int tileId = (row - 1) * columns + col;
+                        resultStorage.reset();
+                        aInt32.setValue(tileId);
+                        intSerde.serialize(aInt32, resultStorage.getDataOutput());
+                        result.set(resultStorage);
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index ad2819f..427e7ee 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -63,6 +63,7 @@
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlStddevPopDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlSumDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSqlVarPopAggregateDescriptor;
@@ -73,6 +74,7 @@
 import org.apache.asterix.runtime.aggregates.scalar.ScalarStddevPopDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSumAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarSumDistinctAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.scalar.ScalarUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarVarDistinctAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.scalar.ScalarVarPopAggregateDescriptor;
@@ -158,11 +160,13 @@
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalSqlUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.GlobalUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateAvgAggregateDescriptor;
@@ -178,11 +182,13 @@
 import org.apache.asterix.runtime.aggregates.std.IntermediateSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateSqlStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateSqlUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateSqlVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.IntermediateUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.IntermediateVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.KurtosisAggregateDescriptor;
@@ -200,11 +206,13 @@
 import org.apache.asterix.runtime.aggregates.std.LocalSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSqlUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.MaxAggregateDescriptor;
@@ -220,11 +228,13 @@
 import org.apache.asterix.runtime.aggregates.std.SqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlStddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlSumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.SqlUnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlVarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SqlVarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.StddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.StddevPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.SumAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.UnionMbrAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.VarAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.VarPopAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
@@ -338,6 +348,7 @@
 import org.apache.asterix.runtime.evaluators.functions.DeepEqualityDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FullTextContainsFunctionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.FullTextContainsWithoutOptionFunctionDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.GetIntersectionDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetItemDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetJobParameterByNameDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.GetTypeDescriptor;
@@ -414,6 +425,7 @@
 import org.apache.asterix.runtime.evaluators.functions.OrDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.ReferenceTileDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
@@ -590,6 +602,7 @@
 import org.apache.asterix.runtime.runningaggregates.std.WinPartitionLenRunningAggregateDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.RangeDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.ScanCollectionDescriptor;
+import org.apache.asterix.runtime.unnestingfunctions.std.SpatialTileDescriptor;
 import org.apache.asterix.runtime.unnestingfunctions.std.SubsetCollectionDescriptor;
 
 /**
@@ -641,6 +654,7 @@
         fc.add(ScanCollectionDescriptor.FACTORY);
         fc.add(RangeDescriptor.FACTORY);
         fc.add(SubsetCollectionDescriptor.FACTORY);
+        fc.add(SpatialTileDescriptor.FACTORY);
 
         // aggregate functions
         fc.add(ListifyAggregateDescriptor.FACTORY);
@@ -693,6 +707,10 @@
         fc.add(EmptyStreamAggregateDescriptor.FACTORY);
         fc.add(NonEmptyStreamAggregateDescriptor.FACTORY);
         fc.add(NullWriterAggregateDescriptor.FACTORY);
+        fc.add(UnionMbrAggregateDescriptor.FACTORY);
+        fc.add(LocalUnionMbrAggregateDescriptor.FACTORY);
+        fc.add(IntermediateUnionMbrAggregateDescriptor.FACTORY);
+        fc.add(GlobalUnionMbrAggregateDescriptor.FACTORY);
 
         // serializable aggregates
         fc.add(SerializableCountAggregateDescriptor.FACTORY);
@@ -754,6 +772,7 @@
         fc.add(ScalarKurtosisDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarSkewnessAggregateDescriptor.FACTORY);
         fc.add(ScalarSkewnessDistinctAggregateDescriptor.FACTORY);
+        fc.add(ScalarUnionMbrAggregateDescriptor.FACTORY);
 
         // SQL aggregates
         fc.add(SqlCountAggregateDescriptor.FACTORY);
@@ -797,6 +816,10 @@
         fc.add(LocalSqlSkewnessAggregateDescriptor.FACTORY);
         fc.add(IntermediateSqlSkewnessAggregateDescriptor.FACTORY);
         fc.add(GlobalSqlSkewnessAggregateDescriptor.FACTORY);
+        fc.add(SqlUnionMbrAggregateDescriptor.FACTORY);
+        fc.add(LocalSqlUnionMbrAggregateDescriptor.FACTORY);
+        fc.add(IntermediateSqlUnionMbrAggregateDescriptor.FACTORY);
+        fc.add(GlobalSqlUnionMbrAggregateDescriptor.FACTORY);
 
         // SQL serializable aggregates
         fc.add(SerializableSqlCountAggregateDescriptor.FACTORY);
@@ -856,6 +879,7 @@
         fc.add(ScalarSqlKurtosisDistinctAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlSkewnessAggregateDescriptor.FACTORY);
         fc.add(ScalarSqlSkewnessDistinctAggregateDescriptor.FACTORY);
+        fc.add(ScalarSqlUnionMbrAggregateDescriptor.FACTORY);
 
         // window functions
         fc.add(DenseRankRunningAggregateDescriptor.FACTORY);
@@ -1101,6 +1125,8 @@
         fc.add(CircleRadiusAccessor.FACTORY);
         fc.add(CircleCenterAccessor.FACTORY);
         fc.add(LineRectanglePolygonAccessor.FACTORY);
+        fc.add(ReferenceTileDescriptor.FACTORY);
+        fc.add(GetIntersectionDescriptor.FACTORY);
 
         // full-text function
         fc.add(FullTextContainsFunctionDescriptor.FACTORY);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java
new file mode 100644
index 0000000..f804d60
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/PlaneSweepJoinOperatorDescriptor.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.spatial;
+
+import java.nio.ByteBuffer;
+
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtilFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivity;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class PlaneSweepJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    private static final int JOIN_BUILD_ACTIVITY_ID = 0;
+    private static final int JOIN_PROBE_ACTIVITY_ID = 1;
+    private final int[] buildKeys;
+    private final int[] probeKeys;
+    private final int memoryForJoin;
+    private final ISpatialJoinUtilFactory imjcf;
+
+    public PlaneSweepJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memoryForJoin, int[] buildKeys,
+            int[] probeKeys, RecordDescriptor recordDescriptor, ISpatialJoinUtilFactory imjcf) {
+        super(spec, 2, 1);
+        outRecDescs[0] = recordDescriptor;
+        this.buildKeys = buildKeys;
+        this.probeKeys = probeKeys;
+        this.memoryForJoin = memoryForJoin;
+        this.imjcf = imjcf;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ActivityId buildAid = new ActivityId(odId, JOIN_BUILD_ACTIVITY_ID);
+        ActivityId probeAid = new ActivityId(odId, JOIN_PROBE_ACTIVITY_ID);
+
+        IActivity probeAN = new JoinProbeActivityNode(probeAid);
+        IActivity buildAN = new JoinBuildActivityNode(buildAid, probeAid);
+
+        builder.addActivity(this, buildAN);
+        builder.addSourceEdge(0, buildAN, 0);
+
+        builder.addActivity(this, probeAN);
+        builder.addSourceEdge(1, probeAN, 0);
+        builder.addTargetEdge(0, probeAN, 0);
+        builder.addBlockingEdge(buildAN, probeAN);
+    }
+
+    public static class JoinCacheTaskState extends AbstractStateObject {
+        private SpatialJoiner joiner;
+
+        private JoinCacheTaskState(JobId jobId, TaskId taskId) {
+            super(jobId, taskId);
+        }
+    }
+
+    private class JoinBuildActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private final ActivityId nljAid;
+
+        public JoinBuildActivityNode(ActivityId id, ActivityId nljAid) {
+            super(id);
+            this.nljAid = nljAid;
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+            final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+
+            return new AbstractUnaryInputSinkOperatorNodePushable() {
+                private JoinCacheTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    state = new JoinCacheTaskState(ctx.getJobletContext().getJobId(),
+                            new TaskId(getActivityId(), partition));
+
+                    ISpatialJoinUtil imjc = imjcf.createSpatialJoinUtil(buildKeys, probeKeys, ctx, nPartitions);
+
+                    state.joiner = new SpatialJoiner(ctx, memoryForJoin, imjc, buildKeys, probeKeys, rd0, rd1);
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    ByteBuffer copyBuffer = ctx.allocateFrame(buffer.capacity());
+                    FrameUtils.copyAndFlip(buffer, copyBuffer);
+                    state.joiner.processBuildFrame(copyBuffer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    state.joiner.processBuildClose();
+                    ctx.setStateObject(state);
+                }
+
+                @Override
+                public void fail() {
+                    // No variables to update.
+                }
+            };
+        }
+    }
+
+    private class JoinProbeActivityNode extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        public JoinProbeActivityNode(ActivityId id) {
+            super(id);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
+            return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+                private JoinCacheTaskState state;
+
+                @Override
+                public void open() throws HyracksDataException {
+                    writer.open();
+                    state = (JoinCacheTaskState) ctx.getStateObject(
+                            new TaskId(new ActivityId(getOperatorId(), JOIN_BUILD_ACTIVITY_ID), partition));
+                }
+
+                @Override
+                public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                    state.joiner.processProbeFrame(buffer, writer);
+                }
+
+                @Override
+                public void close() throws HyracksDataException {
+                    try {
+                        state.joiner.processProbeClose(writer);
+                    } finally {
+                        writer.close();
+                    }
+                }
+
+                @Override
+                public void fail() throws HyracksDataException {
+                    writer.fail();
+                }
+            };
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java
new file mode 100644
index 0000000..aaac216
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/SpatialJoiner.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.spatial;
+
+import java.nio.ByteBuffer;
+import java.util.LinkedList;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.FrameTupleCursor;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFilePointer;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.RunFileStream;
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.TuplePointerCursor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.memory.SpatialSideTuple;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.std.buffermanager.DeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeallocatableFramePool;
+import org.apache.hyracks.dataflow.std.buffermanager.IDeletableTupleBufferManager;
+import org.apache.hyracks.dataflow.std.buffermanager.VariableDeletableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public class SpatialJoiner {
+    private final IDeallocatableFramePool framePool;
+    private final IDeletableTupleBufferManager bufferManager;
+    private final TuplePointerCursor memoryCursor;
+    private final LinkedList<TuplePointer> memoryBuffer = new LinkedList<>();
+
+    private final RunFileStream runFileStream;
+    private final RunFilePointer runFilePointer;
+
+    private SpatialSideTuple memoryTuple;
+    private SpatialSideTuple[] inputTuple;
+    private TuplePointer tp;
+
+    private final ISpatialJoinUtil mjc;
+
+    protected static final int JOIN_PARTITIONS = 2;
+    protected static final int BUILD_PARTITION = 0;
+    protected static final int PROBE_PARTITION = 1;
+
+    protected final IFrame[] inputBuffer;
+    protected final FrameTupleAppender resultAppender;
+    protected final FrameTupleCursor[] inputCursor;
+
+    public SpatialJoiner(IHyracksTaskContext ctx, int memorySize, ISpatialJoinUtil mjc, int[] buildKeys,
+            int[] probeKeys, RecordDescriptor buildRd, RecordDescriptor probeRd) throws HyracksDataException {
+        this.mjc = mjc;
+
+        // Memory (probe buffer)
+        if (memorySize < 5) {
+            throw new RuntimeException(
+                    "SpatialJoiner does not have enough memory (needs > 4, got " + memorySize + ").");
+        }
+
+        inputCursor = new FrameTupleCursor[JOIN_PARTITIONS];
+        inputCursor[BUILD_PARTITION] = new FrameTupleCursor(buildRd);
+        inputCursor[PROBE_PARTITION] = new FrameTupleCursor(probeRd);
+
+        inputBuffer = new IFrame[JOIN_PARTITIONS];
+        inputBuffer[BUILD_PARTITION] = new VSizeFrame(ctx);
+        inputBuffer[PROBE_PARTITION] = new VSizeFrame(ctx);
+
+        //Two frames are used for the runfile stream, and one frame for each input (2 outputs).
+        framePool = new DeallocatableFramePool(ctx, (memorySize - 4) * ctx.getInitialFrameSize());
+        bufferManager = new VariableDeletableTupleMemoryManager(framePool, probeRd);
+        memoryCursor = new TuplePointerCursor(bufferManager.createTuplePointerAccessor());
+
+        // Run File and frame cache (build buffer)
+        runFileStream = new RunFileStream(ctx, "sj-build");
+        runFilePointer = new RunFilePointer();
+        runFileStream.createRunFileWriting();
+        runFileStream.startRunFileWriting();
+
+        memoryTuple = new SpatialSideTuple(mjc, memoryCursor, probeKeys);
+
+        inputTuple = new SpatialSideTuple[JOIN_PARTITIONS];
+        inputTuple[PROBE_PARTITION] = new SpatialSideTuple(mjc, inputCursor[PROBE_PARTITION], probeKeys);
+        inputTuple[BUILD_PARTITION] = new SpatialSideTuple(mjc, inputCursor[BUILD_PARTITION], buildKeys);
+
+        // Result
+        this.resultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+    }
+
+    public void processBuildFrame(ByteBuffer buffer) throws HyracksDataException {
+        inputCursor[BUILD_PARTITION].reset(buffer);
+        for (int x = 0; x < inputCursor[BUILD_PARTITION].getAccessor().getTupleCount(); x++) {
+            runFileStream.addToRunFile(inputCursor[BUILD_PARTITION].getAccessor(), x);
+        }
+    }
+
+    public void processBuildClose() throws HyracksDataException {
+        runFileStream.flushRunFile();
+        runFileStream.startReadingRunFile(inputCursor[BUILD_PARTITION]);
+    }
+
+    public void processProbeFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException {
+        inputCursor[PROBE_PARTITION].reset(buffer);
+        while (buildHasNext() && inputCursor[PROBE_PARTITION].hasNext()) {
+            if (inputCursor[PROBE_PARTITION].hasNext() && mjc.checkToLoadNextProbeTuple(
+                    inputCursor[BUILD_PARTITION].getAccessor(), inputCursor[BUILD_PARTITION].getTupleId() + 1,
+                    inputCursor[PROBE_PARTITION].getAccessor(), inputCursor[PROBE_PARTITION].getTupleId() + 1)) {
+                // Process probe side from stream
+                inputCursor[PROBE_PARTITION].next();
+                processProbeTuple(writer);
+            } else {
+                // Process build side from runfile
+                inputCursor[BUILD_PARTITION].next();
+                processBuildTuple(writer);
+            }
+        }
+    }
+
+    public void processProbeClose(IFrameWriter writer) throws HyracksDataException {
+        while (buildHasNext() && memoryHasTuples()) {
+            // Process build side from runfile
+            inputCursor[BUILD_PARTITION].next();
+            processBuildTuple(writer);
+        }
+        resultAppender.write(writer, true);
+        runFileStream.close();
+        runFileStream.removeRunFile();
+    }
+
+    private boolean buildHasNext() throws HyracksDataException {
+        if (!inputCursor[BUILD_PARTITION].hasNext()) {
+            // Must keep condition in a separate `if` due to actions applied in loadNextBuffer.
+            return runFileStream.loadNextBuffer(inputCursor[BUILD_PARTITION]);
+        } else {
+            return true;
+        }
+    }
+
+    private void processBuildTuple(IFrameWriter writer) throws HyracksDataException {
+        // Check against memory
+        if (memoryHasTuples()) {
+            memoryCursor.reset(memoryBuffer.iterator());
+            while (memoryCursor.hasNext()) {
+                memoryCursor.next();
+                if (inputTuple[BUILD_PARTITION].removeFromMemory(memoryTuple)) {
+                    // remove from memory
+                    bufferManager.deleteTuple(memoryCursor.getTuplePointer());
+                    memoryCursor.remove();
+                    continue;
+                } else if (inputTuple[BUILD_PARTITION].checkForEarlyExit(memoryTuple)) {
+                    // No more possible comparisons
+                    break;
+                } else if (inputTuple[BUILD_PARTITION].compareJoin(memoryTuple)) {
+                    // add to result
+                    addToResult(inputCursor[BUILD_PARTITION].getAccessor(), inputCursor[BUILD_PARTITION].getTupleId(),
+                            memoryCursor.getAccessor(), memoryCursor.getTupleId(), writer);
+                }
+            }
+        }
+    }
+
+    private void processProbeTuple(IFrameWriter writer) throws HyracksDataException {
+        // append to memory
+        // BUILD Cursor is guaranteed to have next
+        if (mjc.checkToSaveInMemory(inputCursor[BUILD_PARTITION].getAccessor(),
+                inputCursor[BUILD_PARTITION].getTupleId() + 1, inputCursor[PROBE_PARTITION].getAccessor(),
+                inputCursor[PROBE_PARTITION].getTupleId())) {
+            if (!addToMemory(inputCursor[PROBE_PARTITION].getAccessor(), inputCursor[PROBE_PARTITION].getTupleId())) {
+                unfreezeAndClearMemory(writer);
+                if (!addToMemory(inputCursor[PROBE_PARTITION].getAccessor(),
+                        inputCursor[PROBE_PARTITION].getTupleId())) {
+                    throw new RuntimeException("Should Never get called.");
+                }
+            }
+        }
+    }
+
+    private void unfreezeAndClearMemory(IFrameWriter writer) throws HyracksDataException {
+        runFilePointer.reset(runFileStream.getReadPointer(), inputCursor[BUILD_PARTITION].getTupleId());
+        while (buildHasNext() && memoryHasTuples()) {
+            // Process build side from runfile
+            inputCursor[BUILD_PARTITION].next();
+            processBuildTuple(writer);
+        }
+        // Clear memory
+        memoryBuffer.clear();
+        bufferManager.reset();
+        // Start reading
+        runFileStream.startReadingRunFile(inputCursor[BUILD_PARTITION], runFilePointer.getFileOffset());
+        inputCursor[BUILD_PARTITION].resetPosition(runFilePointer.getTupleIndex());
+    }
+
+    private boolean addToMemory(IFrameTupleAccessor accessor, int tupleId) throws HyracksDataException {
+        tp = new TuplePointer();
+        if (bufferManager.insertTuple(accessor, tupleId, tp)) {
+            memoryBuffer.add(tp);
+            return true;
+        }
+        return false;
+    }
+
+    private void addToResult(IFrameTupleAccessor buildAccessor, int buildTupleId, IFrameTupleAccessor probeAccessor,
+            int probeTupleId, IFrameWriter writer) throws HyracksDataException {
+        FrameUtils.appendConcatToWriter(writer, resultAppender, buildAccessor, buildTupleId, probeAccessor,
+                probeTupleId);
+    }
+
+    private boolean memoryHasTuples() {
+        return bufferManager.getNumTuples() > 0;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java
new file mode 100644
index 0000000..7c18480
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtil.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.spatial.utils;
+
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISpatialJoinUtil {
+
+    /**
+     * Check to see if the right tuple should be added to memory during the merge join.
+     * The memory is used to check the right tuple with the remaining left tuples.
+     * The check is true if the next left tuple could still match with this right tuple.
+     *
+     * @param buildAccessor
+     * @param probeAccessor
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkToSaveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+    /**
+     * Check to see if the right tuple should be removed from memory during the merge join.
+     * The memory is used to check the right tuple with the remaining left tuples.
+     * The check is true if the next left tuple is NOT able match with this right tuple.
+     *
+     * @param buildAccessor
+     * @param probeAccessor
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkToRemoveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+    /**
+     * Check to see if tuples match join condition
+     * The check is true if they match.
+     *
+     * @param buildAccessor
+     * @param buildTupleIndex
+     * @param probeAccessor
+     * @param probeTupleIndex
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkToSaveInResult(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+
+    /**
+     * Check to see if the spatial object matches the join condition.
+     * The check is true if it matches.
+     *
+     * @param rectBuild
+     * @param rectProbe
+     * @return boolean
+     */
+    boolean compareRectangle(ARectangle rectBuild, ARectangle rectProbe);
+
+    /**
+     * Check to see if the left tuple should stop checking for matches.
+     * The check is true if there can be no more matches
+     *
+     * @param buildAccessor
+     * @param probeAccessor
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkForEarlyExit(IFrameTupleAccessor buildAccessor, int buildTupleIndex, IFrameTupleAccessor probeAccessor,
+            int probeTupleIndex) throws HyracksDataException;
+
+    /**
+     * Check if next tuple should be loaded into memory.
+     * The check is true if there are more tuples
+     *
+     * @param buildAccessor
+     * @param probeAccessor
+     * @return boolean
+     * @throws HyracksDataException
+     */
+    boolean checkToLoadNextProbeTuple(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java
new file mode 100644
index 0000000..0ad911a
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/ISpatialJoinUtilFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.spatial.utils;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ISpatialJoinUtilFactory extends Serializable {
+
+    ISpatialJoinUtil createSpatialJoinUtil(int[] buildKey, int[] probeKey, IHyracksTaskContext ctx, int nPartitions)
+            throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java
new file mode 100644
index 0000000..8ef8a85
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtil.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.spatial.utils;
+
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.memory.SpatialJoinUtil;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntersectSpatialJoinUtil implements ISpatialJoinUtil {
+
+    protected final int[] idBuild;
+    protected final int[] idProbe;
+
+    public IntersectSpatialJoinUtil(int[] idBuild, int[] idProbe) {
+        this.idBuild = idBuild;
+        this.idProbe = idProbe;
+    }
+
+    /**
+     * Right (second argument) interval starts before left (first argument) interval ends.
+     */
+    @Override
+    public boolean checkToSaveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+        int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+        int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+        double buildXmin = SpatialJoinUtil.getRectangleXmin(buildAccessor, buildTupleIndex, idBuild[1]);
+        double probeXmax = SpatialJoinUtil.getRectangleXmax(probeAccessor, probeTupleIndex, idProbe[1]);
+
+        if (buildTileId == probeTileId) {
+            return buildXmin < probeXmax;
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Left (first argument) interval starts after the Right (second argument) interval ends.
+     */
+    @Override
+    public boolean checkToRemoveInMemory(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+        int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+        int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+        double buildXmin = SpatialJoinUtil.getRectangleXmin(buildAccessor, buildTupleIndex, idBuild[1]);
+        double probeXmax = SpatialJoinUtil.getRectangleXmax(probeAccessor, probeTupleIndex, idProbe[1]);
+
+        if (buildTileId != probeTileId) {
+            return true;
+        } else {
+            return buildXmin >= probeXmax;
+        }
+    }
+
+    @Override
+    public boolean checkToSaveInResult(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+        int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+        int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+        ARectangle rectBuild = SpatialJoinUtil.getRectangle(buildAccessor, buildTupleIndex, idBuild[1]);
+        ARectangle rectProbe = SpatialJoinUtil.getRectangle(probeAccessor, probeTupleIndex, idProbe[1]);
+
+        if (buildTileId == probeTileId) {
+            return compareRectangle(rectBuild, rectProbe);
+        } else {
+            return false;
+        }
+    }
+
+    /**
+     * Right (second argument) rectangle starts before left (first argument) rectangle ends.
+     */
+    @Override
+    public boolean checkForEarlyExit(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+        int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+        int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+        double probeXmin = SpatialJoinUtil.getRectangleXmin(probeAccessor, probeTupleIndex, idProbe[1]);
+        double buildXmax = SpatialJoinUtil.getRectangleXmax(buildAccessor, buildTupleIndex, idBuild[1]);
+
+        if (buildTileId != probeTileId) {
+            return true;
+        } else {
+            return buildXmax < probeXmin;
+        }
+    }
+
+    @Override
+    public boolean compareRectangle(ARectangle rectBuild, ARectangle rectProbe) {
+        return SpatialUtils.intersects(rectBuild, rectProbe);
+    }
+
+    @Override
+    public boolean checkToLoadNextProbeTuple(IFrameTupleAccessor buildAccessor, int buildTupleIndex,
+            IFrameTupleAccessor probeAccessor, int probeTupleIndex) throws HyracksDataException {
+        int buildTileId = SpatialJoinUtil.getTileId(buildAccessor, buildTupleIndex, idBuild[0]);
+        int probeTileId = SpatialJoinUtil.getTileId(probeAccessor, probeTupleIndex, idProbe[0]);
+        double probeXmin = SpatialJoinUtil.getRectangleXmin(probeAccessor, probeTupleIndex, idProbe[1]);
+        double buildXmax = SpatialJoinUtil.getRectangleXmax(buildAccessor, buildTupleIndex, idBuild[1]);
+
+        if (buildTileId == probeTileId) {
+            return buildXmax > probeXmin;
+        }
+        return buildTileId >= probeTileId;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java
new file mode 100644
index 0000000..02a79d4
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/IntersectSpatialJoinUtilFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.spatial.utils;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class IntersectSpatialJoinUtilFactory implements ISpatialJoinUtilFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ISpatialJoinUtil createSpatialJoinUtil(int[] buildKeys, int[] probeKeys, IHyracksTaskContext ctx,
+            int nPartitions) throws HyracksDataException {
+        return new IntersectSpatialJoinUtil(buildKeys, probeKeys);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java
new file mode 100644
index 0000000..f8c2619
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialJoinUtil.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.spatial.utils.memory;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.om.base.APoint;
+import org.apache.asterix.om.base.ARectangle;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SpatialJoinUtil {
+
+    private SpatialJoinUtil() {
+    }
+
+    public static ARectangle getRectangle(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+            throws HyracksDataException {
+        int start = getFieldOffset(accessor, tupleId, fieldId);
+        double xmin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+        double ymin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+        double xmax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+        double ymax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+        return new ARectangle(new APoint(xmin, ymin), new APoint(xmax, ymax));
+    }
+
+    public static int getTileId(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+        int start = getFieldOffset(accessor, tupleId, fieldId);
+        int tileId = AInt32SerializerDeserializer.getInt(accessor.getBuffer().array(), start);
+        return tileId;
+    }
+
+    public static double getRectangleXmin(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+            throws HyracksDataException {
+        int start = getFieldOffset(accessor, tupleId, fieldId);
+        double xmin = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+        return xmin;
+    }
+
+    public static double getRectangleXmax(IFrameTupleAccessor accessor, int tupleId, int fieldId)
+            throws HyracksDataException {
+        int start = getFieldOffset(accessor, tupleId, fieldId);
+        double xmax = ADoubleSerializerDeserializer.getDouble(accessor.getBuffer().array(),
+                start + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+        return xmax;
+    }
+
+    public static int getFieldOffset(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+        return getFieldOffsetWithTag(accessor, tupleId, fieldId) + 1;
+    }
+
+    public static int getFieldOffsetWithTag(IFrameTupleAccessor accessor, int tupleId, int fieldId) {
+        int start = accessor.getTupleStartOffset(tupleId) + accessor.getFieldSlotsLength()
+                + accessor.getFieldStartOffset(tupleId, fieldId);
+        return start;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java
new file mode 100644
index 0000000..db84749
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/joins/spatial/utils/memory/SpatialSideTuple.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.operators.joins.spatial.utils.memory;
+
+import org.apache.asterix.runtime.operators.joins.interval.utils.memory.ITupleCursor;
+import org.apache.asterix.runtime.operators.joins.spatial.utils.ISpatialJoinUtil;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SpatialSideTuple {
+    // Tuple access
+    int[] fieldId;
+    ITupleCursor cursor;
+
+    // Join details
+    final ISpatialJoinUtil imjc;
+
+    public SpatialSideTuple(ISpatialJoinUtil imjc, ITupleCursor cursor, int[] fieldId) {
+        this.imjc = imjc;
+        this.cursor = cursor;
+        this.fieldId = fieldId;
+    }
+
+    public int getTupleIndex() {
+        return cursor.getTupleId();
+    }
+
+    public ITupleCursor getCursor() {
+        return cursor;
+    }
+
+    public boolean compareJoin(SpatialSideTuple ist) throws HyracksDataException {
+        return imjc.checkToSaveInResult(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+                ist.cursor.getTupleId());
+    }
+
+    public boolean removeFromMemory(SpatialSideTuple ist) throws HyracksDataException {
+        return imjc.checkToRemoveInMemory(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+                ist.cursor.getTupleId());
+    }
+
+    public boolean checkForEarlyExit(SpatialSideTuple ist) throws HyracksDataException {
+        return imjc.checkForEarlyExit(cursor.getAccessor(), cursor.getTupleId(), ist.cursor.getAccessor(),
+                ist.cursor.getTupleId());
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java
new file mode 100644
index 0000000..c16fc59
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/unnestingfunctions/std/SpatialTileDescriptor.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.unnestingfunctions.std;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.Coordinate;
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class SpatialTileDescriptor extends AbstractUnnestingFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new SpatialTileDescriptor();
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SPATIAL_TILE;
+    }
+
+    @Override
+    public IUnnestingEvaluatorFactory createUnnestingEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IUnnestingEvaluatorFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IUnnestingEvaluator createUnnestingEvaluator(final IEvaluatorContext ctx)
+                    throws HyracksDataException {
+                final IHyracksTaskContext hyracksTaskContext = ctx.getTaskContext();
+
+                return new IUnnestingEvaluator() {
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private List<Integer> tileValues = new ArrayList<>();
+                    private final IPointable inputArg0 = new VoidPointable();
+                    private final IPointable inputArg1 = new VoidPointable();
+                    private final IPointable inputArg2 = new VoidPointable();
+                    private final IPointable inputArg3 = new VoidPointable();
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval1 = args[1].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval2 = args[2].createScalarEvaluator(ctx);
+                    private final IScalarEvaluator eval3 = args[3].createScalarEvaluator(ctx);
+
+                    private final AMutableInt32 aInt32 = new AMutableInt32(0);
+                    int pos;
+
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer intSerde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+                    @Override
+                    public void init(IFrameTupleReference tuple) throws HyracksDataException {
+                        eval0.evaluate(tuple, inputArg0);
+                        eval1.evaluate(tuple, inputArg1);
+                        eval2.evaluate(tuple, inputArg2);
+                        eval3.evaluate(tuple, inputArg3);
+
+                        byte[] bytes0 = inputArg0.getByteArray();
+                        byte[] bytes1 = inputArg1.getByteArray();
+                        byte[] bytes2 = inputArg2.getByteArray();
+                        byte[] bytes3 = inputArg3.getByteArray();
+
+                        int offset0 = inputArg0.getStartOffset();
+                        int offset1 = inputArg1.getStartOffset();
+                        int offset2 = inputArg2.getStartOffset();
+                        int offset3 = inputArg3.getStartOffset();
+
+                        ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes0[offset0]);
+                        ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes1[offset1]);
+                        ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes2[offset2]);
+                        ATypeTag tag3 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes3[offset3]);
+
+                        if (tag0 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes0[offset0],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+                        if (tag1 != ATypeTag.RECTANGLE) {
+                            throw new TypeMismatchException(sourceLoc, getIdentifier(), 0, bytes1[offset1],
+                                    ATypeTag.SERIALIZED_RECTANGLE_TYPE_TAG);
+                        }
+
+                        double x1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double y1 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double x2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+                        double y2 = ADoubleSerializerDeserializer.getDouble(bytes0, offset0 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+                        double minX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.X));
+                        double minY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getBottomLeftCoordinateOffset(Coordinate.Y));
+
+                        double maxX = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.X));
+                        double maxY = ADoubleSerializerDeserializer.getDouble(bytes1, offset1 + 1
+                                + ARectangleSerializerDeserializer.getUpperRightCoordinateOffset(Coordinate.Y));
+
+                        int rows = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes2, offset2);
+                        int columns = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes3, offset3);
+
+                        // Unnest iff two input datasets overlap, which means partitioning MBR is not [(0,0),(0,0)]
+                        tileValues.clear();
+                        pos = 0;
+                        if (!((minX == 0.0) && (minY == 0.0) && (maxX == 0.0) && (maxY == 0.0))) {
+                            // Unnest iff the record overlaps with the partitioning MBR
+                            if (!((x1 > maxX) || (minX > x2) || (y1 > maxY) || (minY > y2))) {
+                                int row1 = (int) Math.ceil((y1 - minY) * rows / (maxY - minY));
+                                int col1 = (int) Math.ceil((x1 - minX) * columns / (maxX - minX));
+                                int row2 = (int) Math.ceil((y2 - minY) * rows / (maxY - minY));
+                                int col2 = (int) Math.ceil((x2 - minX) * columns / (maxX - minX));
+
+                                row1 = Math.min(Math.max(1, row1), rows * columns);
+                                col1 = Math.min(Math.max(1, col1), rows * columns);
+                                row2 = Math.min(Math.max(1, row2), rows * columns);
+                                col2 = Math.min(Math.max(1, col2), rows * columns);
+
+                                int minRow = Math.min(row1, row2);
+                                int maxRow = Math.max(row1, row2);
+                                int minCol = Math.min(col1, col2);
+                                int maxCol = Math.max(col1, col2);
+
+                                for (int i = minRow; i <= maxRow; i++) {
+                                    for (int j = minCol; j <= maxCol; j++) {
+                                        int tileId = (i - 1) * columns + j;
+                                        tileValues.add(tileId);
+                                    }
+                                }
+                            }
+                        }
+                    }
+
+                    @SuppressWarnings("unchecked")
+                    @Override
+                    public boolean step(IPointable result) throws HyracksDataException {
+                        if (pos < tileValues.size()) {
+                            aInt32.setValue(tileValues.get(pos));
+                            resultStorage.reset();
+                            intSerde.serialize(aInt32, resultStorage.getDataOutput());
+                            result.set(resultStorage);
+                            ++pos;
+                            return true;
+                        }
+                        return false;
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index d590f71..4a900af 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -66,6 +66,7 @@
     SINK_WRITE,
     SORT_GROUP_BY,
     SORT_MERGE_EXCHANGE,
+    SPATIAL_JOIN,
     SPLIT,
     STABLE_SORT,
     STATS,
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index f17fa70..625418a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -495,8 +495,8 @@
 
     @Override
     public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
-        addIndent(indent)
-                .append("forward: range-map = " + op.getSideDataExpression().getValue().accept(exprVisitor, indent));
+        addIndent(indent).append(
+                "forward: shared-variable = " + op.getSideDataExpression().getValue().accept(exprVisitor, indent));
         return null;
     }