You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sedona.apache.org by ji...@apache.org on 2022/10/18 18:02:34 UTC

[incubator-sedona] branch master updated: [SEDONA-177] Extend RDD API for RangeQuery JoinQuery to support spatial predicates other than coveredBy and intersects. (#700)

This is an automated email from the ASF dual-hosted git repository.

jiayu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-sedona.git


The following commit(s) were added to refs/heads/master by this push:
     new b2ee308e [SEDONA-177] Extend RDD API for RangeQuery JoinQuery to support spatial predicates other than coveredBy and intersects. (#700)
b2ee308e is described below

commit b2ee308efecbf04acec54f396b9fa3d1da3876fc
Author: Kristin Cowalcijk <mo...@yeah.net>
AuthorDate: Wed Oct 19 02:02:27 2022 +0800

    [SEDONA-177] Extend RDD API for RangeQuery JoinQuery to support spatial predicates other than coveredBy and intersects. (#700)
---
 .../joinJudgement/DynamicIndexLookupJudgement.java |   6 +-
 .../core/joinJudgement/JoinConditionMatcher.java   | 118 +++
 .../sedona/core/joinJudgement/JudgementBase.java   |  16 +-
 .../sedona/core/joinJudgement/JudgementHelper.java |  58 --
 .../joinJudgement/LeftIndexLookupJudgement.java    |   5 +-
 .../core/joinJudgement/NestedLoopJudgement.java    |   5 +-
 .../joinJudgement/RightIndexLookupJudgement.java   |   5 +-
 .../sedona/core/rangeJudgement/JudgementBase.java  |  39 +-
 .../sedona/core/rangeJudgement/RangeFilter.java    |  13 +-
 .../core/rangeJudgement/RangeFilterUsingIndex.java |  17 +-
 .../sedona/core/spatialOperator/JoinQuery.java     | 226 ++++-
 .../sedona/core/spatialOperator/RangeQuery.java    |  75 +-
 .../core/spatialOperator/SpatialPredicate.java     |  57 ++
 .../SpatialPredicateEvaluators.java                | 125 +++
 .../ScalaEarthdataMapperRunnableExample.scala      |   5 +-
 .../apache/sedona/core/showcase/ScalaExample.scala |  19 +-
 .../sedona/core/spatialOperator/JoinQueryTest.java | 313 +++++++
 .../core/spatialOperator/RangeQueryTest.java       | 130 +++
 .../core/spatialOperator/SpatialQueryTestBase.java | 134 +++
 .../test/resources/spatial-join-query-window.tsv   | 207 +++++
 .../resources/spatial-predicates-test-data.tsv     | 907 +++++++++++++++++++++
 .../scala/org/apache/sedona/core/scalaTest.scala   |  32 +
 docs/tutorial/rdd.md                               |  38 +-
 .../sedona/flink/expressions/Predicates.java       |  18 +-
 python/tests/sql/test_predicate_join.py            |   6 +-
 .../strategy/join/BroadcastIndexJoinExec.scala     |  23 +-
 .../strategy/join/DistanceJoinExec.scala           |   3 +-
 .../strategy/join/JoinQueryDetector.scala          |  78 +-
 .../sedona_sql/strategy/join/RangeJoinExec.scala   |   3 +-
 .../strategy/join/TraitJoinQueryExec.scala         |   5 +-
 .../org/apache/sedona/sql/SpatialJoinSuite.scala   | 163 ++++
 .../org/apache/sedona/sql/TestBaseScala.scala      |   2 +
 .../apache/sedona/sql/predicateJoinTestScala.scala |   6 +-
 33 files changed, 2612 insertions(+), 245 deletions(-)

diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/DynamicIndexLookupJudgement.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/DynamicIndexLookupJudgement.java
index c1fbcf59..e3cae32d 100644
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/DynamicIndexLookupJudgement.java
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/DynamicIndexLookupJudgement.java
@@ -26,6 +26,7 @@ import org.apache.log4j.Logger;
 import org.apache.sedona.core.enums.IndexType;
 import org.apache.sedona.core.enums.JoinBuildSide;
 import org.apache.sedona.core.monitoring.Metric;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.sedona.core.utils.TimeUtils;
 import org.apache.spark.TaskContext;
 import org.apache.spark.api.java.function.FlatMapFunction2;
@@ -61,7 +62,8 @@ public class DynamicIndexLookupJudgement<T extends Geometry, U extends Geometry>
     /**
      * @see JudgementBase
      */
-    public DynamicIndexLookupJudgement(boolean considerBoundaryIntersection,
+    public DynamicIndexLookupJudgement(
+            SpatialPredicate spatialPredicate,
             IndexType indexType,
             JoinBuildSide joinBuildSide,
             @Nullable DedupParams dedupParams,
@@ -70,7 +72,7 @@ public class DynamicIndexLookupJudgement<T extends Geometry, U extends Geometry>
             Metric resultCount,
             Metric candidateCount)
     {
-        super(considerBoundaryIntersection, dedupParams);
+        super(spatialPredicate, dedupParams);
         this.indexType = indexType;
         this.joinBuildSide = joinBuildSide;
         this.buildCount = buildCount;
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/JoinConditionMatcher.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/JoinConditionMatcher.java
new file mode 100644
index 00000000..b9679ceb
--- /dev/null
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/JoinConditionMatcher.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sedona.core.joinJudgement;
+
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
+import org.apache.sedona.core.spatialOperator.SpatialPredicateEvaluators;
+import org.apache.sedona.core.utils.HalfOpenRectangle;
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryFactory;
+import org.locationtech.jts.geom.Point;
+
+/**
+ * Match geometries using join condition with a mechanism for deduplication.
+ */
+public abstract class JoinConditionMatcher implements SpatialPredicateEvaluators.SpatialPredicateEvaluator {
+
+    public boolean match(Geometry left, Geometry right, HalfOpenRectangle extent) {
+        if (extent != null) {
+            // Handle easy case: points. Since each point is assigned to exactly one partition,
+            // different partitions cannot emit duplicate results.
+            if (left instanceof Point || right instanceof Point) {
+                return eval(left, right);
+            }
+
+            // Neither geometry is a point
+
+            // Check if reference point of the intersection of the bounding boxes lies within
+            // the extent of this partition. If not, don't run any checks. Let the partition
+            // that contains the reference point do all the work.
+            Envelope intersection =
+                    left.getEnvelopeInternal().intersection(right.getEnvelopeInternal());
+            if (!intersection.isNull()) {
+                final Point referencePoint =
+                        makePoint(intersection.getMinX(), intersection.getMinY(), left.getFactory());
+                if (!extent.contains(referencePoint)) {
+                    return false;
+                }
+            }
+        }
+
+        return eval(left, right);
+    }
+
+    private static Point makePoint(double x, double y, GeometryFactory factory) {
+        return factory.createPoint(new Coordinate(x, y));
+    }
+
+    public static JoinConditionMatcher create(SpatialPredicate predicate) {
+        switch (predicate) {
+            case CONTAINS:
+                return new ContainsMatcher();
+            case INTERSECTS:
+                return new IntersectsMatcher();
+            case WITHIN:
+                return new WithinMatcher();
+            case COVERS:
+                return new CoversMatcher();
+            case COVERED_BY:
+                return new CoveredByMatcher();
+            case TOUCHES:
+                return new TouchesMatcher();
+            case OVERLAPS:
+                return new OverlapsMatcher();
+            case CROSSES:
+                return new CrossesMatcher();
+            case EQUALS:
+                return new EqualsMatcher();
+            default:
+                throw new IllegalArgumentException("Invalid spatial predicate: " + predicate);
+        }
+    }
+
+    private static class ContainsMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.ContainsEvaluator {}
+
+    private static class IntersectsMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.IntersectsEvaluator {}
+
+    private static class WithinMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.WithinEvaluator {}
+
+    private static class CoversMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.CoversEvaluator {}
+
+    private static class CoveredByMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.CoveredByEvaluator {}
+
+    private static class TouchesMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.TouchesEvaluator {}
+
+    private static class OverlapsMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.OverlapsEvaluator {}
+
+    private static class CrossesMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.CrossesEvaluator {}
+
+    private static class EqualsMatcher extends JoinConditionMatcher
+            implements SpatialPredicateEvaluators.EqualsEvaluator {}
+}
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementBase.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementBase.java
index a3f2394e..17493b8e 100644
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementBase.java
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementBase.java
@@ -21,16 +21,14 @@ package org.apache.sedona.core.joinJudgement;
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.sedona.core.utils.HalfOpenRectangle;
 import org.apache.spark.SparkContext;
 import org.apache.spark.TaskContext;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.broadcast.Broadcast;
-import org.locationtech.jts.geom.Coordinate;
 import org.locationtech.jts.geom.Envelope;
 import org.locationtech.jts.geom.Geometry;
-import org.locationtech.jts.geom.GeometryFactory;
-import org.locationtech.jts.geom.Point;
 
 import javax.annotation.Nullable;
 
@@ -65,20 +63,21 @@ abstract class JudgementBase
 
     private static final Logger log = LogManager.getLogger(JudgementBase.class);
 
-    private final boolean considerBoundaryIntersection;
+    private final SpatialPredicate spatialPredicate;
     // Supplier will return a broadcasted reference if broadcastDedupParams() is called,
     // otherwise a local reference is returned.
     private SerializableSupplier<DedupParams> dedupParams;
 
     transient private HalfOpenRectangle extent;
+    transient private JoinConditionMatcher matcher;
 
     /**
-     * @param considerBoundaryIntersection true for 'intersects', false for 'contains' join condition
+     * @param spatialPredicate spatial predicate as join condition
      * @param dedupParams Optional information to activate de-dup logic
      */
-    protected JudgementBase(boolean considerBoundaryIntersection, @Nullable DedupParams dedupParams)
+    protected JudgementBase(SpatialPredicate spatialPredicate, @Nullable DedupParams dedupParams)
     {
-        this.considerBoundaryIntersection = considerBoundaryIntersection;
+        this.spatialPredicate = spatialPredicate;
         this.dedupParams = dedupParams == null ? null : () -> dedupParams;
     }
 
@@ -120,10 +119,11 @@ abstract class JudgementBase
         else {
             log.warn("Didn't find partition extent for this partition: " + partitionId);
         }
+        matcher = JoinConditionMatcher.create(spatialPredicate);
     }
 
     public boolean match(Geometry left, Geometry right)
     {
-        return JudgementHelper.match(left, right, extent, considerBoundaryIntersection);
+        return matcher.match(left, right, extent);
     }
 }
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementHelper.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementHelper.java
deleted file mode 100644
index 273ccdee..00000000
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/JudgementHelper.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT 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.sedona.core.joinJudgement;
-
-import org.apache.sedona.core.utils.HalfOpenRectangle;
-import org.locationtech.jts.geom.*;
-
-public class JudgementHelper {
-    public static boolean match(Geometry left, Geometry right, HalfOpenRectangle extent, boolean considerBoundaryIntersection)
-    {
-        if (extent != null) {
-            // Handle easy case: points. Since each point is assigned to exactly one partition,
-            // different partitions cannot emit duplicate results.
-            if (left instanceof Point || right instanceof Point) {
-                return geoMatch(left, right, considerBoundaryIntersection);
-            }
-
-            // Neither geometry is a point
-
-            // Check if reference point of the intersection of the bounding boxes lies within
-            // the extent of this partition. If not, don't run any checks. Let the partition
-            // that contains the reference point do all the work.
-            Envelope intersection =
-                    left.getEnvelopeInternal().intersection(right.getEnvelopeInternal());
-            if (!intersection.isNull()) {
-                final Point referencePoint =
-                        makePoint(intersection.getMinX(), intersection.getMinY(), left.getFactory());
-                if (!extent.contains(referencePoint)) {
-                    return false;
-                }
-            }
-        }
-
-        return geoMatch(left, right, considerBoundaryIntersection);
-    }
-
-    private static Point makePoint(double x, double y, GeometryFactory factory)
-    {
-        return factory.createPoint(new Coordinate(x, y));
-    }
-
-    private static boolean geoMatch(Geometry left, Geometry right, boolean considerBoundaryIntersection)
-    {
-        //log.warn("Check "+left.toText()+" with "+right.toText());
-        return considerBoundaryIntersection ? left.intersects(right) : left.covers(right);
-    }
-}
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/LeftIndexLookupJudgement.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/LeftIndexLookupJudgement.java
index ca23696c..9114c5f5 100644
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/LeftIndexLookupJudgement.java
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/LeftIndexLookupJudgement.java
@@ -20,6 +20,7 @@
 package org.apache.sedona.core.joinJudgement;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.spark.api.java.function.FlatMapFunction2;
 import org.locationtech.jts.geom.Geometry;
 import org.locationtech.jts.index.SpatialIndex;
@@ -39,9 +40,9 @@ public class LeftIndexLookupJudgement<T extends Geometry, U extends Geometry>
     /**
      * @see JudgementBase
      */
-    public LeftIndexLookupJudgement(boolean considerBoundaryIntersection, @Nullable DedupParams dedupParams)
+    public LeftIndexLookupJudgement(SpatialPredicate spatialPredicate, @Nullable DedupParams dedupParams)
     {
-        super(considerBoundaryIntersection, dedupParams);
+        super(spatialPredicate, dedupParams);
     }
 
     @Override
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/NestedLoopJudgement.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/NestedLoopJudgement.java
index adc25dab..0774c14a 100644
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/NestedLoopJudgement.java
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/NestedLoopJudgement.java
@@ -22,6 +22,7 @@ package org.apache.sedona.core.joinJudgement;
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.spark.api.java.function.FlatMapFunction2;
 import org.locationtech.jts.geom.Geometry;
 
@@ -42,9 +43,9 @@ public class NestedLoopJudgement<T extends Geometry, U extends Geometry>
     /**
      * @see JudgementBase
      */
-    public NestedLoopJudgement(boolean considerBoundaryIntersection, @Nullable DedupParams dedupParams)
+    public NestedLoopJudgement(SpatialPredicate spatialPredicate, @Nullable DedupParams dedupParams)
     {
-        super(considerBoundaryIntersection, dedupParams);
+        super(spatialPredicate, dedupParams);
     }
 
     @Override
diff --git a/core/src/main/java/org/apache/sedona/core/joinJudgement/RightIndexLookupJudgement.java b/core/src/main/java/org/apache/sedona/core/joinJudgement/RightIndexLookupJudgement.java
index be9c63d7..b47274eb 100644
--- a/core/src/main/java/org/apache/sedona/core/joinJudgement/RightIndexLookupJudgement.java
+++ b/core/src/main/java/org/apache/sedona/core/joinJudgement/RightIndexLookupJudgement.java
@@ -20,6 +20,7 @@
 package org.apache.sedona.core.joinJudgement;
 
 import org.apache.commons.lang3.tuple.Pair;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.spark.api.java.function.FlatMapFunction2;
 import org.locationtech.jts.geom.Geometry;
 import org.locationtech.jts.index.SpatialIndex;
@@ -39,9 +40,9 @@ public class RightIndexLookupJudgement<T extends Geometry, U extends Geometry>
     /**
      * @see JudgementBase
      */
-    public RightIndexLookupJudgement(boolean considerBoundaryIntersection, @Nullable DedupParams dedupParams)
+    public RightIndexLookupJudgement(SpatialPredicate spatialPredicate, @Nullable DedupParams dedupParams)
     {
-        super(considerBoundaryIntersection, dedupParams);
+        super(spatialPredicate, dedupParams);
     }
 
     @Override
diff --git a/core/src/main/java/org/apache/sedona/core/rangeJudgement/JudgementBase.java b/core/src/main/java/org/apache/sedona/core/rangeJudgement/JudgementBase.java
index f76d5358..3186fc67 100644
--- a/core/src/main/java/org/apache/sedona/core/rangeJudgement/JudgementBase.java
+++ b/core/src/main/java/org/apache/sedona/core/rangeJudgement/JudgementBase.java
@@ -19,8 +19,8 @@
 
 package org.apache.sedona.core.rangeJudgement;
 
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
+import org.apache.sedona.core.spatialOperator.SpatialPredicateEvaluators;
 import org.locationtech.jts.geom.Geometry;
 
 import java.io.Serializable;
@@ -28,32 +28,45 @@ import java.io.Serializable;
 public class JudgementBase<U extends Geometry>
         implements Serializable
 {
-
-    private static final Logger log = LogManager.getLogger(JudgementBase.class);
-    private final boolean considerBoundaryIntersection;
-    protected boolean leftCoveredByRight = true;
+    private final SpatialPredicateEvaluators.SpatialPredicateEvaluator evaluator;
     U queryGeometry;
 
+    /**
+     * Instantiates a new range filter using index.
+     *
+     * @param queryWindow the query window
+     * @param spatialPredicate spatial predicate in query criteria {@code geom <spatialPredicate> queryWindow}
+     */
+    public JudgementBase(U queryWindow, SpatialPredicate spatialPredicate)
+    {
+        this.queryGeometry = queryWindow;
+        this.evaluator = SpatialPredicateEvaluators.create(spatialPredicate);
+    }
+
     /**
      * Instantiates a new range filter using index.
      *
      * @param queryWindow the query window
      * @param considerBoundaryIntersection the consider boundary intersection
+     * @param leftCoveredByRight query window covered by geometry, or query window covers geometry.
+     *                           only effective when {@code considerBoundaryIntersection} was false
      */
     public JudgementBase(U queryWindow, boolean considerBoundaryIntersection, boolean leftCoveredByRight)
     {
-        this.considerBoundaryIntersection = considerBoundaryIntersection;
-        this.queryGeometry = queryWindow;
-        this.leftCoveredByRight = leftCoveredByRight;
+        this(queryWindow, resolveSpatialPredicate(considerBoundaryIntersection, leftCoveredByRight));
     }
 
     public boolean match(Geometry spatialObject, Geometry queryWindow)
+    {
+        return evaluator.eval(spatialObject, queryWindow);
+    }
+
+    public static SpatialPredicate resolveSpatialPredicate(boolean considerBoundaryIntersection, boolean leftCoveredByRight)
     {
         if (considerBoundaryIntersection) {
-            return queryWindow.intersects(spatialObject);
-        }
-        else {
-            return queryWindow.covers(spatialObject);
+            return SpatialPredicate.INTERSECTS;
+        } else {
+            return leftCoveredByRight? SpatialPredicate.COVERED_BY: SpatialPredicate.COVERS;
         }
     }
 }
diff --git a/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilter.java b/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilter.java
index 33eca2da..7ef06fe2 100644
--- a/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilter.java
+++ b/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilter.java
@@ -19,6 +19,7 @@
 
 package org.apache.sedona.core.rangeJudgement;
 
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.spark.api.java.function.Function;
 import org.locationtech.jts.geom.Geometry;
 
@@ -29,6 +30,11 @@ public class RangeFilter<U extends Geometry, T extends Geometry>
         implements Function<T, Boolean>
 {
 
+    public RangeFilter(U queryWindow, SpatialPredicate spatialPredicate)
+    {
+        super(queryWindow, spatialPredicate);
+    }
+
     public RangeFilter(U queryWindow, boolean considerBoundaryIntersection, boolean leftCoveredByRight)
     {
         super(queryWindow, considerBoundaryIntersection, leftCoveredByRight);
@@ -40,11 +46,6 @@ public class RangeFilter<U extends Geometry, T extends Geometry>
     public Boolean call(T geometry)
             throws Exception
     {
-        if (leftCoveredByRight) {
-            return match(geometry, queryGeometry);
-        }
-        else {
-            return match(queryGeometry, geometry);
-        }
+        return match(geometry, queryGeometry);
     }
 }
diff --git a/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilterUsingIndex.java b/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilterUsingIndex.java
index 9c32204a..75b4148b 100644
--- a/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilterUsingIndex.java
+++ b/core/src/main/java/org/apache/sedona/core/rangeJudgement/RangeFilterUsingIndex.java
@@ -19,6 +19,7 @@
 
 package org.apache.sedona.core.rangeJudgement;
 
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.spark.api.java.function.FlatMapFunction;
 import org.locationtech.jts.geom.Geometry;
 import org.locationtech.jts.index.SpatialIndex;
@@ -34,6 +35,11 @@ public class RangeFilterUsingIndex<U extends Geometry, T extends Geometry>
         implements FlatMapFunction<Iterator<SpatialIndex>, T>
 {
 
+    public RangeFilterUsingIndex(U queryWindow, SpatialPredicate spatialPredicate)
+    {
+        super(queryWindow, spatialPredicate);
+    }
+
     public RangeFilterUsingIndex(U queryWindow, boolean considerBoundaryIntersection, boolean leftCoveredByRight)
     {
         super(queryWindow, considerBoundaryIntersection, leftCoveredByRight);
@@ -58,15 +64,8 @@ public class RangeFilterUsingIndex<U extends Geometry, T extends Geometry>
         List<T> results = new ArrayList<T>();
         List<T> tempResults = treeIndex.query(this.queryGeometry.getEnvelopeInternal());
         for (T tempResult : tempResults) {
-            if (leftCoveredByRight) {
-                if (match(tempResult, queryGeometry)) {
-                    results.add(tempResult);
-                }
-            }
-            else {
-                if (match(queryGeometry, tempResult)) {
-                    results.add(tempResult);
-                }
+            if (match(tempResult, queryGeometry)) {
+                results.add(tempResult);
             }
         }
         return results.iterator();
diff --git a/core/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java b/core/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java
index 07b81bf4..ff4f130e 100644
--- a/core/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java
+++ b/core/src/main/java/org/apache/sedona/core/spatialOperator/JoinQuery.java
@@ -128,10 +128,39 @@ public class JoinQuery
     }
 
     /**
-     * Inner joins two sets of geometries on 'contains' or 'intersects' relationship.
+     * Inner joins two sets of geometries on specified spatial predicate.
+     * <p>
+     * {@code spatialPredicate} is the spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}
+     * <p>
+     * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching geometries.
+     * <p>
+     * If {@code useIndex} is true, the join scans query windows and uses an index of geometries
+     * built prior to invoking the join to lookup matches.
+     * <p>
+     * Duplicate geometries present in the input queryWindowRDD, regardless of their non-spatial attributes, will not be reflected in the join results.
+     * Duplicate geometries present in the input spatialRDD, regardless of their non-spatial attributes, will be reflected in the join results.
+     * @param <U> Type of the geometries in queryWindowRDD set
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries which serve as query windows
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}
+     * @return RDD of pairs where each pair contains a geometry and a set of matching geometries
+     * @throws Exception the exception
+     */
+    public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, List<T>> SpatialJoinQuery(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        final JoinParams joinParams = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
+        return collectGeometriesByKey(joinResults);
+    }
+
+    /**
+     * Inner joins two sets of geometries on 'covers' or 'intersects' relationship.
      * <p>
      * If {@code considerBoundaryIntersection} is {@code true}, returns pairs of geometries
-     * which intersect. Otherwise, returns pairs of geometries where first geometry contains second geometry.
+     * which intersect. Otherwise, returns pairs of geometries where first geometry covers second geometry.
      * <p>
      * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching geometries.
      * <p>
@@ -145,10 +174,11 @@ public class JoinQuery
      * @param spatialRDD Set of geometries
      * @param queryRDD Set of geometries which serve as query windows
      * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
-     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'contains' otherwise
+     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'covers' otherwise
      * @return RDD of pairs where each pair contains a geometry and a set of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, List<T>> SpatialJoinQuery(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -165,10 +195,38 @@ public class JoinQuery
     }
 
     /**
-     * Inner joins two sets of geometries on 'contains' or 'intersects' relationship. Results are put in a flat pair format.
+     * Inner joins two sets of geometries on specified spatial predicate. Results are put in a flat pair format.
+     * <p>
+     * {@code spatialPredicate} is the spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}.
+     * <p>
+     * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching geometries.
+     * <p>
+     * If {@code useIndex} is true, the join scans query windows and uses an index of geometries
+     * built prior to invoking the join to lookup matches.
+     * <p>
+     * Duplicates present in the input RDDs will be reflected in the join results.
+     *
+     * @param <U> Type of the geometries in queryWindowRDD set
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries which serve as query windows
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}
+     * @return RDD of pairs of matching geometries
+     * @throws Exception the exception
+     */
+    public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, T> SpatialJoinQueryFlat(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        final JoinParams params = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        return spatialJoin(queryRDD, spatialRDD, params);
+    }
+
+    /**
+     * Inner joins two sets of geometries on 'covers' or 'intersects' relationship. Results are put in a flat pair format.
      * <p>
      * If {@code considerBoundaryIntersection} is {@code true}, returns pairs of geometries
-     * which intersect. Otherwise, returns pairs of geometries where first geometry contains second geometry.
+     * which intersect. Otherwise, returns pairs of geometries where first geometry covers second geometry.
      * <p>
      * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching geometries.
      * <p>
@@ -182,10 +240,11 @@ public class JoinQuery
      * @param spatialRDD Set of geometries
      * @param queryRDD Set of geometries which serve as query windows
      * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
-     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'contains' otherwise
+     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'covers' otherwise
      * @return RDD of pairs of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, T> SpatialJoinQueryFlat(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -199,6 +258,29 @@ public class JoinQuery
         return spatialJoin(queryRDD, spatialRDD, joinParams);
     }
 
+    /**
+     * {@link #SpatialJoinQueryFlat(SpatialRDD, SpatialRDD, boolean, SpatialPredicate)} count by key.
+     * <p>
+     * Duplicate geometries present in the input queryWindowRDD RDD, regardless of their non-spatial attributes, will not be reflected in the join results.
+     * Duplicate geometries present in the input spatialRDD RDD, regardless of their non-spatial attributes, will be reflected in the join results.
+     *
+     * @param <U> Type of the geometries in queryWindowRDD set
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries which serve as query windows
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}
+     * @return the result of {@link #SpatialJoinQueryFlat(SpatialRDD, SpatialRDD, boolean, SpatialPredicate)}, but in this pair RDD, each pair contains a geometry and the count of matching geometries
+     * @throws Exception the exception
+     */
+    public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, Long> SpatialJoinQueryCountByKey(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        final JoinParams joinParams = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
+        return countGeometriesByKey(joinResults);
+    }
+
     /**
      * {@link #SpatialJoinQueryFlat(SpatialRDD, SpatialRDD, boolean, boolean)} count by key.
      * <p>
@@ -210,10 +292,11 @@ public class JoinQuery
      * @param spatialRDD Set of geometries
      * @param queryRDD Set of geometries which serve as query windows
      * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
-     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'contains' otherwise
+     * @param considerBoundaryIntersection Join relationship type: 'intersects' if true, 'covers' otherwise
      * @return the result of {@link #SpatialJoinQueryFlat(SpatialRDD, SpatialRDD, boolean, boolean)}, but in this pair RDD, each pair contains a geometry and the count of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, Long> SpatialJoinQueryCountByKey(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -230,10 +313,40 @@ public class JoinQuery
     }
 
     /**
-     * Inner joins two sets of geometries on 'within' relationship (aka. distance join). Results are put in a flat pair format.
+     * Inner joins two sets of geometries, where the query windows are circles (aka. distance join). Results are put in a flat pair format.
+     * <p>
+     * {@code spatialPredicate} is the spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}.
+     * <p>
+     * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching circle/geometry.
+     * <p>
+     * If {@code useIndex} is true, the join scans circles and uses an index of geometries
+     * built prior to invoking the join to lookup matches.
+     * <p>
+     * Duplicates present in the input RDDs will be reflected in the join results.
+     *
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}, should be one of {@code INTERSECTS} and {@code COVERED_BY}
+     * @return RDD of pairs of matching geometries
+     * @throws Exception the exception
+     */
+    public static <T extends Geometry> JavaPairRDD<Geometry, T> DistanceJoinQueryFlat(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        if (spatialPredicate != SpatialPredicate.COVERED_BY && spatialPredicate != SpatialPredicate.INTERSECTS) {
+            throw new IllegalArgumentException("Spatial predicate for distance join should be one of INTERSECTS and COVERED_BY");
+        }
+        final JoinParams joinParams = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        return distanceJoin(spatialRDD, queryRDD, joinParams);
+    }
+
+    /**
+     * Inner joins two sets of geometries on 'coveredBy' relationship (aka. distance join). Results are put in a flat pair format.
      * <p>
      * If {@code considerBoundaryIntersection} is {@code true}, returns pairs of circle/geometry
-     * which intersect. Otherwise, returns pairs of geometries where first circle contains second geometry.
+     * which intersect. Otherwise, returns pairs of geometries where first circle covers second geometry.
      * <p>
      * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching circle/geometry.
      * <p>
@@ -250,6 +363,7 @@ public class JoinQuery
      * @return RDD of pairs of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <T extends Geometry> JavaPairRDD<Geometry, T> DistanceJoinQueryFlat(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -264,12 +378,45 @@ public class JoinQuery
     }
 
     /**
-     * Inner joins two sets of geometries on 'within' relationship (aka. distance join).
+     * Inner joins two sets of geometries, where the query windows are circles (aka. distance join).
+     * The query window objects are converted to circle objects. The radius is the given distance.
+     * Eventually, the original window objects are recovered and outputted.
+     * <p>
+     * {@code spatialPredicate} is the spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}.
+     * <p>
+     * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching circle/geometry.
+     * <p>
+     * If {@code useIndex} is true, the join scans circles and uses an index of geometries
+     * built prior to invoking the join to lookup matches.
+     * <p>
+     * Duplicate geometries present in the input CircleRDD, regardless of their non-spatial attributes, will not be reflected in the join results.
+     * Duplicate geometries present in the input spatialRDD, regardless of their non-spatial attributes, will be reflected in the join results.
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}, should be one of {@code INTERSECTS} and {@code COVERED_BY}
+     * @return RDD of pairs where each pair contains a geometry and a set of matching geometries
+     * @throws Exception the exception
+     */
+    public static <T extends Geometry> JavaPairRDD<Geometry, List<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        if (spatialPredicate != SpatialPredicate.COVERED_BY && spatialPredicate != SpatialPredicate.INTERSECTS) {
+            throw new IllegalArgumentException("Spatial predicate for distance join should be one of INTERSECTS and COVERED_BY");
+        }
+        final JoinParams joinParams = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
+        return collectGeometriesByKey(joinResults);
+    }
+
+    /**
+     * Inner joins two sets of geometries on 'coveredBy' relationship (aka. distance join).
      * The query window objects are converted to circle objects. The radius is the given distance.
      * Eventually, the original window objects are recovered and outputted.
      * <p>
      * If {@code considerBoundaryIntersection} is {@code true}, returns pairs of circle/geometry
-     * which intersect. Otherwise, returns pairs of geometries where first circle contains second geometry.
+     * which intersect. Otherwise, returns pairs of geometries where first circle covers second geometry.
      * <p>
      * If {@code useIndex} is false, the join uses nested loop algorithm to identify matching circle/geometry.
      * <p>
@@ -286,6 +433,7 @@ public class JoinQuery
      * @return RDD of pairs where each pair contains a geometry and a set of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <T extends Geometry> JavaPairRDD<Geometry, List<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -301,6 +449,30 @@ public class JoinQuery
         return collectGeometriesByKey(joinResults);
     }
 
+    /**
+     * {@link #DistanceJoinQueryFlat(SpatialRDD, CircleRDD, boolean, boolean)} count by key.
+     * <p>
+     * Duplicate geometries present in the input CircleRDD, regardless of their non-spatial attributes, will not be reflected in the join results.
+     * Duplicate geometries present in the input spatialRDD, regardless of their non-spatial attributes, will be reflected in the join results.
+     * @param <T> Type of the geometries in spatialRDD set
+     * @param spatialRDD Set of geometries
+     * @param queryRDD Set of geometries
+     * @param useIndex Boolean indicating whether the join should use the index from {@code spatialRDD.indexedRDD}
+     * @param spatialPredicate Spatial predicate in join condition {@code spatialRDD <spatialPredicate> queryRDD}, should be one of {@code INTERSECTS} and {@code COVERED_BY}
+     * @return the result of {@link #DistanceJoinQueryFlat(SpatialRDD, CircleRDD, boolean, boolean)}, but in this pair RDD, each pair contains a geometry and the count of matching geometries
+     * @throws Exception the exception
+     */
+    public static <T extends Geometry> JavaPairRDD<Geometry, Long> DistanceJoinQueryCountByKey(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, SpatialPredicate spatialPredicate)
+            throws Exception
+    {
+        if (spatialPredicate != SpatialPredicate.COVERED_BY && spatialPredicate != SpatialPredicate.INTERSECTS) {
+            throw new IllegalArgumentException("Spatial predicate for distance join should be one of INTERSECTS and COVERED_BY");
+        }
+        final JoinParams joinParams = new JoinParams(useIndex, SpatialPredicate.inverse(spatialPredicate));
+        final JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
+        return countGeometriesByKey(joinResults);
+    }
+
     /**
      * {@link #DistanceJoinQueryFlat(SpatialRDD, CircleRDD, boolean, boolean)} count by key.
      * <p>
@@ -314,6 +486,7 @@ public class JoinQuery
      * @return the result of {@link #DistanceJoinQueryFlat(SpatialRDD, CircleRDD, boolean, boolean)}, but in this pair RDD, each pair contains a geometry and the count of matching geometries
      * @throws Exception the exception
      */
+    @Deprecated
     public static <T extends Geometry> JavaPairRDD<Geometry, Long> DistanceJoinQueryCountByKey(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
             throws Exception
     {
@@ -379,13 +552,13 @@ public class JoinQuery
         if (joinParams.useIndex) {
             if (rightRDD.indexedRDD != null) {
                 final RightIndexLookupJudgement judgement =
-                        new RightIndexLookupJudgement(joinParams.considerBoundaryIntersection, dedupParams);
+                        new RightIndexLookupJudgement(joinParams.spatialPredicate, dedupParams);
                 judgement.broadcastDedupParams(cxt);
                 joinResult = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.indexedRDD, judgement);
             }
             else if (leftRDD.indexedRDD != null) {
                 final LeftIndexLookupJudgement judgement =
-                        new LeftIndexLookupJudgement(joinParams.considerBoundaryIntersection, dedupParams);
+                        new LeftIndexLookupJudgement(joinParams.spatialPredicate, dedupParams);
                 judgement.broadcastDedupParams(cxt);
                 joinResult = leftRDD.indexedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
             }
@@ -393,7 +566,7 @@ public class JoinQuery
                 log.warn("UseIndex is true, but no index exists. Will build index on the fly.");
                 DynamicIndexLookupJudgement judgement =
                         new DynamicIndexLookupJudgement(
-                                joinParams.considerBoundaryIntersection,
+                                joinParams.spatialPredicate,
                                 joinParams.indexType,
                                 joinParams.joinBuildSide,
                                 dedupParams,
@@ -403,7 +576,7 @@ public class JoinQuery
             }
         }
         else {
-            NestedLoopJudgement judgement = new NestedLoopJudgement(joinParams.considerBoundaryIntersection, dedupParams);
+            NestedLoopJudgement judgement = new NestedLoopJudgement(joinParams.spatialPredicate, dedupParams);
             judgement.broadcastDedupParams(cxt);
             joinResult = rightRDD.spatialPartitionedRDD.zipPartitions(leftRDD.spatialPartitionedRDD, judgement);
         }
@@ -422,22 +595,33 @@ public class JoinQuery
     public static final class JoinParams
     {
         public final boolean useIndex;
-        public final boolean considerBoundaryIntersection;
+        public final SpatialPredicate spatialPredicate;
         public final IndexType indexType;
         public final JoinBuildSide joinBuildSide;
 
+        public JoinParams(boolean useIndex, SpatialPredicate spatialPredicate, IndexType polygonIndexType, JoinBuildSide joinBuildSide)
+        {
+            this.useIndex = useIndex;
+            this.spatialPredicate = spatialPredicate;
+            this.indexType = polygonIndexType;
+            this.joinBuildSide = joinBuildSide;
+        }
+
+        public JoinParams(boolean useIndex, SpatialPredicate spatialPredicate)
+        {
+            this(useIndex, spatialPredicate, IndexType.RTREE, JoinBuildSide.RIGHT);
+        }
+
+        @Deprecated
         public JoinParams(boolean useIndex, boolean considerBoundaryIntersection)
         {
             this(useIndex, considerBoundaryIntersection, IndexType.RTREE, JoinBuildSide.RIGHT);
         }
 
+        @Deprecated
         public JoinParams(boolean useIndex, boolean considerBoundaryIntersection, IndexType polygonIndexType, JoinBuildSide joinBuildSide)
         {
-            this.useIndex = useIndex;
-            this.considerBoundaryIntersection = considerBoundaryIntersection;
-            this.indexType = polygonIndexType;
-            this.joinBuildSide = joinBuildSide;
+            this(useIndex, considerBoundaryIntersection? SpatialPredicate.INTERSECTS: SpatialPredicate.COVERS, polygonIndexType, joinBuildSide);
         }
     }
 }
-
diff --git a/core/src/main/java/org/apache/sedona/core/spatialOperator/RangeQuery.java b/core/src/main/java/org/apache/sedona/core/spatialOperator/RangeQuery.java
index c502c183..746d2409 100644
--- a/core/src/main/java/org/apache/sedona/core/spatialOperator/RangeQuery.java
+++ b/core/src/main/java/org/apache/sedona/core/spatialOperator/RangeQuery.java
@@ -45,12 +45,12 @@ public class RangeQuery
      *
      * @param spatialRDD the spatial RDD
      * @param originalQueryGeometry the original query window
-     * @param considerBoundaryIntersection the consider boundary intersection
+     * @param spatialPredicate spatial predicate as query criteria {@code geom <spatialPredicate> originalQueryGeometry}
      * @param useIndex the use index
      * @return the java RDD
      * @throws Exception the exception
      */
-    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, U originalQueryGeometry, boolean considerBoundaryIntersection, boolean useIndex)
+    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, U originalQueryGeometry, SpatialPredicate spatialPredicate, boolean useIndex)
             throws Exception
     {
         U queryGeometry = originalQueryGeometry;
@@ -62,10 +62,10 @@ public class RangeQuery
             if (spatialRDD.indexedRawRDD == null) {
                 throw new Exception("[RangeQuery][SpatialRangeQuery] Index doesn't exist. Please build index on rawSpatialRDD.");
             }
-            return spatialRDD.indexedRawRDD.mapPartitions(new RangeFilterUsingIndex(queryGeometry, considerBoundaryIntersection, true));
+            return spatialRDD.indexedRawRDD.mapPartitions(new RangeFilterUsingIndex(queryGeometry, spatialPredicate));
         }
         else {
-            return spatialRDD.getRawSpatialRDD().filter(new RangeFilter(queryGeometry, considerBoundaryIntersection, true));
+            return spatialRDD.getRawSpatialRDD().filter(new RangeFilter(queryGeometry, spatialPredicate));
         }
     }
 
@@ -74,12 +74,12 @@ public class RangeQuery
      *
      * @param spatialRDD the spatial RDD
      * @param queryWindow the original query window
-     * @param considerBoundaryIntersection the consider boundary intersection
+     * @param spatialPredicate spatial predicate as query criteria {@code geom <spatialPredicate> queryWindow}
      * @param useIndex the use index
      * @return the java RDD
      * @throws Exception the exception
      */
-    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, Envelope queryWindow, boolean considerBoundaryIntersection, boolean useIndex)
+    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, Envelope queryWindow, SpatialPredicate spatialPredicate, boolean useIndex)
             throws Exception
     {
         Coordinate[] coordinates = new Coordinate[5];
@@ -90,7 +90,41 @@ public class RangeQuery
         coordinates[4] = coordinates[0];
         GeometryFactory geometryFactory = new GeometryFactory();
         U queryGeometry = (U) geometryFactory.createPolygon(coordinates);
-        return SpatialRangeQuery(spatialRDD, queryGeometry, considerBoundaryIntersection, useIndex);
+        return SpatialRangeQuery(spatialRDD, queryGeometry, spatialPredicate, useIndex);
+    }
+
+    /**
+     * Spatial range query. Return objects in SpatialRDD are covered/intersected by originalQueryGeometry
+     *
+     * @param spatialRDD the spatial RDD
+     * @param originalQueryGeometry the original query window
+     * @param considerBoundaryIntersection the consider boundary intersection
+     * @param useIndex the use index
+     * @return the java RDD
+     * @throws Exception the exception
+     */
+    @Deprecated
+    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, U originalQueryGeometry, boolean considerBoundaryIntersection, boolean useIndex)
+            throws Exception
+    {
+        return SpatialRangeQuery(spatialRDD, originalQueryGeometry, considerBoundaryIntersection? SpatialPredicate.INTERSECTS: SpatialPredicate.COVERED_BY, useIndex);
+    }
+
+    /**
+     * Spatial range query. Return objects in SpatialRDD are covered/intersected by queryWindow/Envelope
+     *
+     * @param spatialRDD the spatial RDD
+     * @param queryWindow the original query window
+     * @param considerBoundaryIntersection the consider boundary intersection
+     * @param useIndex the use index
+     * @return the java RDD
+     * @throws Exception the exception
+     */
+    @Deprecated
+    public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(SpatialRDD<T> spatialRDD, Envelope queryWindow, boolean considerBoundaryIntersection, boolean useIndex)
+            throws Exception
+    {
+        return SpatialRangeQuery(spatialRDD, queryWindow, considerBoundaryIntersection? SpatialPredicate.INTERSECTS: SpatialPredicate.COVERED_BY, useIndex);
     }
 
     /**
@@ -103,18 +137,11 @@ public class RangeQuery
      * @return the java RDD
      * @throws Exception the exception
      */
+    @Deprecated
     public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(Envelope queryWindow, SpatialRDD<T> spatialRDD, boolean considerBoundaryIntersection, boolean useIndex)
             throws Exception
     {
-        Coordinate[] coordinates = new Coordinate[5];
-        coordinates[0] = new Coordinate(queryWindow.getMinX(), queryWindow.getMinY());
-        coordinates[1] = new Coordinate(queryWindow.getMinX(), queryWindow.getMaxY());
-        coordinates[2] = new Coordinate(queryWindow.getMaxX(), queryWindow.getMaxY());
-        coordinates[3] = new Coordinate(queryWindow.getMaxX(), queryWindow.getMinY());
-        coordinates[4] = coordinates[0];
-        GeometryFactory geometryFactory = new GeometryFactory();
-        U queryGeometry = (U) geometryFactory.createPolygon(coordinates);
-        return SpatialRangeQuery(queryGeometry, spatialRDD, considerBoundaryIntersection, useIndex);
+        return SpatialRangeQuery(spatialRDD, queryWindow, considerBoundaryIntersection? SpatialPredicate.INTERSECTS: SpatialPredicate.COVERS, useIndex);
     }
 
     /**
@@ -127,22 +154,10 @@ public class RangeQuery
      * @return the java RDD
      * @throws Exception the exception
      */
+    @Deprecated
     public static <U extends Geometry, T extends Geometry> JavaRDD<T> SpatialRangeQuery(U originalQueryGeometry, SpatialRDD<T> spatialRDD, boolean considerBoundaryIntersection, boolean useIndex)
             throws Exception
     {
-        U queryGeometry = originalQueryGeometry;
-        if (spatialRDD.getCRStransformation()) {
-            queryGeometry = CRSTransformation.Transform(spatialRDD.getSourceEpsgCode(), spatialRDD.getTargetEpgsgCode(), originalQueryGeometry);
-        }
-
-        if (useIndex == true) {
-            if (spatialRDD.indexedRawRDD == null) {
-                throw new Exception("[RangeQuery][SpatialRangeQuery] Index doesn't exist. Please build index on rawSpatialRDD.");
-            }
-            return spatialRDD.indexedRawRDD.mapPartitions(new RangeFilterUsingIndex(queryGeometry, considerBoundaryIntersection, false));
-        }
-        else {
-            return spatialRDD.getRawSpatialRDD().filter(new RangeFilter(queryGeometry, considerBoundaryIntersection, false));
-        }
+        return SpatialRangeQuery(spatialRDD, originalQueryGeometry, considerBoundaryIntersection? SpatialPredicate.INTERSECTS: SpatialPredicate.COVERS, useIndex);
     }
 }
diff --git a/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java b/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java
new file mode 100644
index 00000000..3f506c8c
--- /dev/null
+++ b/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicate.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.sedona.core.spatialOperator;
+
+/**
+ * Spatial predicates for range queries and join queries.
+ * Please refer to <a href="https://en.wikipedia.org/wiki/DE-9IM#Spatial_predicates">Spatial predicates</a>
+ * for the semantics of these spatial predicates.
+ */
+public enum SpatialPredicate {
+    CONTAINS,
+    INTERSECTS,
+    WITHIN,
+    COVERS,
+    COVERED_BY,
+    TOUCHES,
+    OVERLAPS,
+    CROSSES,
+    EQUALS;
+
+    /**
+     * Get inverse predicate of given spatial predicate
+     * @param predicate spatial predicate
+     * @return inverse predicate
+     */
+    public static SpatialPredicate inverse(SpatialPredicate predicate) {
+        switch (predicate) {
+            case CONTAINS:
+                return SpatialPredicate.WITHIN;
+            case WITHIN:
+                return SpatialPredicate.CONTAINS;
+            case COVERS:
+                return SpatialPredicate.COVERED_BY;
+            case COVERED_BY:
+                return SpatialPredicate.COVERS;
+            default:
+                return predicate;
+        }
+    }
+}
diff --git a/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicateEvaluators.java b/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicateEvaluators.java
new file mode 100644
index 00000000..ed39c507
--- /dev/null
+++ b/core/src/main/java/org/apache/sedona/core/spatialOperator/SpatialPredicateEvaluators.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sedona.core.spatialOperator;
+
+import org.locationtech.jts.geom.Geometry;
+
+import java.io.Serializable;
+
+public class SpatialPredicateEvaluators {
+    private SpatialPredicateEvaluators() {}
+
+    /**
+     * SpatialPredicateEvaluator for evaluating spatial predicates, it also works as a trait which will
+     * be mixed into {@link org.apache.sedona.core.joinJudgement.JoinConditionMatcher}.
+     */
+    public interface SpatialPredicateEvaluator extends Serializable {
+        boolean eval(Geometry left, Geometry right);
+    }
+
+    public interface ContainsEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.contains(right);
+        }
+    }
+
+    public interface IntersectsEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.intersects(right);
+        }
+    }
+
+    public interface WithinEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.within(right);
+        }
+    }
+
+    public interface CoversEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.covers(right);
+        }
+    }
+
+    public interface CoveredByEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.coveredBy(right);
+        }
+    }
+
+    public interface TouchesEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.touches(right);
+        }
+    }
+
+    public interface OverlapsEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.overlaps(right);
+        }
+    }
+
+    public interface CrossesEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.crosses(right);
+        }
+    }
+
+    public interface EqualsEvaluator extends SpatialPredicateEvaluator {
+        default boolean eval(Geometry left, Geometry right) {
+            return left.symDifference(right).isEmpty();
+        }
+    }
+
+    private static class ConcreteContainsEvaluator implements ContainsEvaluator {}
+    private static class ConcreteIntersectsEvaluator implements IntersectsEvaluator {}
+    private static class ConcreteWithinEvaluator implements WithinEvaluator {}
+    private static class ConcreteCoversEvaluator implements CoversEvaluator {}
+    private static class ConcreteCoveredByEvaluator implements CoveredByEvaluator {}
+    private static class ConcreteTouchesEvaluator implements TouchesEvaluator {}
+    private static class ConcreteOverlapsEvaluator implements OverlapsEvaluator {}
+    private static class ConcreteCrossesEvaluator implements CrossesEvaluator {}
+    private static class ConcreteEqualsEvaluator implements EqualsEvaluator {}
+
+    public static SpatialPredicateEvaluator create(SpatialPredicate predicate) {
+        switch (predicate) {
+            case CONTAINS:
+                return new ConcreteContainsEvaluator();
+            case INTERSECTS:
+                return new ConcreteIntersectsEvaluator();
+            case WITHIN:
+                return new ConcreteWithinEvaluator();
+            case COVERS:
+                return new ConcreteCoversEvaluator();
+            case COVERED_BY:
+                return new ConcreteCoveredByEvaluator();
+            case TOUCHES:
+                return new ConcreteTouchesEvaluator();
+            case OVERLAPS:
+                return new ConcreteOverlapsEvaluator();
+            case CROSSES:
+                return new ConcreteCrossesEvaluator();
+            case EQUALS:
+                return new ConcreteEqualsEvaluator();
+            default:
+                throw new IllegalArgumentException("Invalid spatial predicate: " + predicate);
+        }
+    }
+}
diff --git a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala
index 1ec10720..64b70239 100644
--- a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala
+++ b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaEarthdataMapperRunnableExample.scala
@@ -23,6 +23,7 @@ import org.apache.log4j.{Level, Logger}
 import org.apache.sedona.core.enums.{FileDataSplitter, IndexType}
 import org.apache.sedona.core.formatMapper.EarthdataHDFPointMapper
 import org.apache.sedona.core.spatialOperator.RangeQuery
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.spatialRDD.PointRDD
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.{SparkConf, SparkContext}
@@ -59,7 +60,7 @@ object ScalaEarthdataMapperRunnableExample extends App {
     var i = 0
     while (i < loopTimes) {
       var resultSize = 0L
-      resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, false).count
+      resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, SpatialPredicate.COVERED_BY, false).count
       i = i + 1
     }
   }
@@ -74,7 +75,7 @@ object ScalaEarthdataMapperRunnableExample extends App {
     var i = 0
     while (i < loopTimes) {
       var resultSize = 0L
-      resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, true).count
+      resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, SpatialPredicate.COVERED_BY, true).count
       i = i + 1
     }
   }
diff --git a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala
index 5478ff3a..9a5dde5a 100644
--- a/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala
+++ b/core/src/main/scala/org/apache/sedona/core/showcase/ScalaExample.scala
@@ -23,6 +23,7 @@ import org.apache.log4j.{Level, Logger}
 import org.apache.sedona.core.enums.{FileDataSplitter, GridType, IndexType}
 import org.apache.sedona.core.formatMapper.shapefileParser.ShapefileRDD
 import org.apache.sedona.core.serde.SedonaKryoRegistrator
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.spatialOperator.{JoinQuery, KNNQuery, RangeQuery}
 import org.apache.sedona.core.spatialRDD.{CircleRDD, PointRDD, PolygonRDD}
 import org.apache.spark.serializer.KryoSerializer
@@ -89,7 +90,7 @@ object ScalaExample extends App {
     val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, true, StorageLevel.MEMORY_ONLY)
     objectRDD.rawSpatialRDD.persist(StorageLevel.MEMORY_ONLY)
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, false, false).count
+      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, false).count
     }
   }
 
@@ -104,7 +105,7 @@ object ScalaExample extends App {
     objectRDD.buildIndex(PointRDDIndexType, false)
     objectRDD.indexedRawRDD.persist(StorageLevel.MEMORY_ONLY)
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, false, true).count
+      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, true).count
     }
 
   }
@@ -151,7 +152,7 @@ object ScalaExample extends App {
     objectRDD.spatialPartitionedRDD.persist(StorageLevel.MEMORY_ONLY)
     queryWindowRDD.spatialPartitionedRDD.persist(StorageLevel.MEMORY_ONLY)
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, false, true).count
+      val resultSize = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, false, SpatialPredicate.INTERSECTS).count
     }
   }
 
@@ -173,7 +174,7 @@ object ScalaExample extends App {
     queryWindowRDD.spatialPartitionedRDD.persist(StorageLevel.MEMORY_ONLY)
 
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, true, false).count()
+      val resultSize = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, true, SpatialPredicate.COVERED_BY).count()
     }
   }
 
@@ -193,7 +194,7 @@ object ScalaExample extends App {
     queryWindowRDD.spatialPartitionedRDD.persist(StorageLevel.MEMORY_ONLY)
 
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = JoinQuery.DistanceJoinQuery(objectRDD, queryWindowRDD, false, true).count()
+      val resultSize = JoinQuery.DistanceJoinQuery(objectRDD, queryWindowRDD, false, SpatialPredicate.INTERSECTS).count()
     }
   }
 
@@ -215,7 +216,7 @@ object ScalaExample extends App {
     queryWindowRDD.spatialPartitionedRDD.persist(StorageLevel.MEMORY_ONLY)
 
     for (i <- 1 to eachQueryLoopTimes) {
-      val resultSize = JoinQuery.DistanceJoinQuery(objectRDD, queryWindowRDD, true, true).count
+      val resultSize = JoinQuery.DistanceJoinQuery(objectRDD, queryWindowRDD, true, SpatialPredicate.INTERSECTS).count
     }
   }
 
@@ -227,7 +228,7 @@ object ScalaExample extends App {
     while ( {
       i < eachQueryLoopTimes
     }) {
-      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, false, false).count
+      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, false).count
       assert(resultSize > -1)
 
       {
@@ -247,7 +248,7 @@ object ScalaExample extends App {
     while ( {
       i < eachQueryLoopTimes
     }) {
-      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, false, true).count
+      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, true).count
       assert(resultSize > -1)
 
       {
@@ -262,7 +263,7 @@ object ScalaExample extends App {
     val shapefileRDD = new ShapefileRDD(sc, ShapeFileInputLocation)
     val spatialRDD = new PolygonRDD(shapefileRDD.getPolygonRDD)
     try
-      RangeQuery.SpatialRangeQuery(spatialRDD, new Envelope(-180, 180, -90, 90), false, false).count
+      RangeQuery.SpatialRangeQuery(spatialRDD, new Envelope(-180, 180, -90, 90), SpatialPredicate.COVERED_BY, false).count
     catch {
       case e: Exception =>
         // TODO Auto-generated catch block
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java
new file mode 100644
index 00000000..59dde388
--- /dev/null
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryTest.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sedona.core.spatialOperator;
+
+import org.apache.sedona.core.enums.GridType;
+import org.apache.sedona.core.enums.IndexType;
+import org.apache.sedona.core.spatialRDD.CircleRDD;
+import org.apache.sedona.core.spatialRDD.SpatialRDD;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.locationtech.jts.geom.Geometry;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@RunWith(Parameterized.class)
+public class JoinQueryTest extends SpatialQueryTestBase {
+
+    @BeforeClass
+    public static void onceExecutedBeforeAll() throws IOException {
+        initialize(JoinQueryTest.class.getSimpleName(), "spatial-predicates-test-data.tsv");
+    }
+
+    @AfterClass
+    public static void teardown() {
+        sc.stop();
+    }
+
+    @Parameterized.Parameters(name = "JoinQueryTest-{index}: {0}")
+    public static SpatialPredicate[] spatialPredicates() {
+        return SpatialPredicate.values();
+    }
+
+    private final SpatialPredicate spatialPredicate;
+    private final JavaRDD<Geometry> queryDataRdd;
+    private final Map<Integer, Geometry> queryWindowDataset;
+    private final Map<Integer, List<Integer>> expectedResults;
+
+    public JoinQueryTest(SpatialPredicate predicate) throws IOException {
+        this.spatialPredicate = predicate;
+        String queryWindowPath = "spatial-join-query-window.tsv";
+        this.queryDataRdd = readTestDataAsRDD(queryWindowPath);
+        this.queryWindowDataset = readTestDataAsMap(queryWindowPath);
+        expectedResults = buildExpectedResults(spatialPredicate);
+        Assert.assertFalse("expected results should not be empty", expectedResults.isEmpty());
+    }
+
+    private Map<Integer, List<Integer>> buildExpectedResults(SpatialPredicate spatialPredicate) {
+        Map<Integer, List<Integer>> results = new HashMap<>();
+        queryWindowDataset.forEach((queryWindowId, queryWindow) -> testDataset.forEach((geomId, value) -> {
+            if (evaluateSpatialPredicate(spatialPredicate, value, queryWindow)) {
+                results.computeIfAbsent(queryWindowId, k -> new ArrayList<>()).add(geomId);
+            }
+        }));
+        results.forEach((k, v) -> v.sort(Integer::compareTo));
+        return results;
+    }
+
+    private Map<Integer, List<Integer>> spatialJoinResultRddToIdMap(JavaPairRDD<Geometry, List<Geometry>> resultRdd) {
+        Map<Integer, List<Integer>> resultIdMap = new HashMap<>();
+        resultRdd.collect().forEach(pair -> {
+            Geometry queryWindow = pair._1;
+            List<Geometry> queryResults = pair._2;
+            int queryWindowId = ((UserData) queryWindow.getUserData()).getId();
+            List<Integer> geomIds = queryResults.stream().map(geom -> ((UserData) geom.getUserData()).getId()).sorted(Integer::compareTo).collect(Collectors.toList());
+            resultIdMap.put(queryWindowId, geomIds);
+        });
+        return resultIdMap;
+    }
+
+    private Map<Integer, List<Integer>> spatialJoinFlatResultRddToIdMap(JavaPairRDD<Geometry, Geometry> resultRdd) {
+        Map<Integer, List<Integer>> resultIdMap = new HashMap<>();
+        resultRdd.collect().forEach(pair -> {
+            Geometry queryWindow = pair._1;
+            Geometry queryResult = pair._2;
+            int queryWindowId = ((UserData) queryWindow.getUserData()).getId();
+            int geomId = ((UserData) queryResult.getUserData()).getId();
+            resultIdMap.computeIfAbsent(queryWindowId, k -> new ArrayList<>()).add(geomId);
+        });
+        resultIdMap.forEach((k, v) -> v.sort(Integer::compareTo));
+        return resultIdMap;
+    }
+
+    @Test
+    public void testSpatialJoinWithoutIndex() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, spatialPredicate);
+        verifySpatialJoinResult(actualResultRdd);
+    }
+
+    @Test
+    public void testSpatialJoinWithSpatialRddIndex() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        spatialRDD.buildIndex(IndexType.RTREE, true);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, true, spatialPredicate);
+        verifySpatialJoinResult(actualResultRdd);
+    }
+
+    @Test
+    public void testSpatialJoinWithQueryWindowIndex() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        queryRDD.buildIndex(IndexType.RTREE, true);
+        JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, true, spatialPredicate);
+        verifySpatialJoinResult(actualResultRdd);
+    }
+
+    @Test
+    public void testSpatialJoinWithDynamicIndex() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, true, spatialPredicate);
+        verifySpatialJoinResult(actualResultRdd);
+    }
+
+    @Test
+    public void testSpatialJoinFlat() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        JavaPairRDD<Geometry, Geometry> actualResultRdd = JoinQuery.SpatialJoinQueryFlat(spatialRDD, queryRDD, false, spatialPredicate);
+        verifySpatialJoinFlatResult(actualResultRdd);
+    }
+
+    @Test
+    public void testSpatialJoinCountByKey() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        JavaPairRDD<Geometry, Long> actualResultRdd = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, false, spatialPredicate);
+        verifySpatialJoinCountByKeyResult(actualResultRdd);
+    }
+
+    @Test
+    public void testDistanceJoin() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        CircleRDD circleRDD = new CircleRDD(queryRDD, 1.0);
+        circleRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        if (spatialPredicate == SpatialPredicate.INTERSECTS || spatialPredicate == SpatialPredicate.COVERED_BY) {
+            boolean considerBoundaryIntersection = spatialPredicate == SpatialPredicate.INTERSECTS;
+            JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.DistanceJoinQuery(spatialRDD, circleRDD, false, spatialPredicate);
+            JavaPairRDD<Geometry, List<Geometry>> expectedResultRdd = JoinQuery.DistanceJoinQuery(spatialRDD, circleRDD, false, considerBoundaryIntersection);
+            verifySpatialJoinResult(expectedResultRdd, actualResultRdd);
+        } else {
+            Assert.assertThrows(IllegalArgumentException.class, () -> JoinQuery.DistanceJoinQuery(spatialRDD, circleRDD, false, spatialPredicate));
+        }
+    }
+
+    @Test
+    public void testDistanceJoinFlat() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        CircleRDD circleRDD = new CircleRDD(queryRDD, 1.0);
+        circleRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        if (spatialPredicate == SpatialPredicate.INTERSECTS || spatialPredicate == SpatialPredicate.COVERED_BY) {
+            boolean considerBoundaryIntersection = spatialPredicate == SpatialPredicate.INTERSECTS;
+            JavaPairRDD<Geometry, Geometry> actualResultRdd = JoinQuery.DistanceJoinQueryFlat(spatialRDD, circleRDD, false, spatialPredicate);
+            JavaPairRDD<Geometry, Geometry> expectedResultRdd = JoinQuery.DistanceJoinQueryFlat(spatialRDD, circleRDD, false, considerBoundaryIntersection);
+            verifySpatialJoinFlatResult(expectedResultRdd, actualResultRdd);
+        } else {
+            Assert.assertThrows(IllegalArgumentException.class, () -> JoinQuery.DistanceJoinQueryFlat(spatialRDD, circleRDD, false, spatialPredicate));
+        }
+    }
+
+    @Test
+    public void testDistanceJoinCountByKey() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        CircleRDD circleRDD = new CircleRDD(queryRDD, 1.0);
+        circleRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        if (spatialPredicate == SpatialPredicate.INTERSECTS || spatialPredicate == SpatialPredicate.COVERED_BY) {
+            boolean considerBoundaryIntersection = spatialPredicate == SpatialPredicate.INTERSECTS;
+            JavaPairRDD<Geometry, Long> actualResultRdd = JoinQuery.DistanceJoinQueryCountByKey(spatialRDD, circleRDD, false, spatialPredicate);
+            JavaPairRDD<Geometry, Long> expectedResultRdd = JoinQuery.DistanceJoinQueryCountByKey(spatialRDD, circleRDD, false, considerBoundaryIntersection);
+            verifySpatialJoinCountByKeyResult(expectedResultRdd, actualResultRdd);
+        } else {
+            Assert.assertThrows(IllegalArgumentException.class, () -> JoinQuery.DistanceJoinQueryCountByKey(spatialRDD, circleRDD, false, spatialPredicate));
+        }
+    }
+
+    @Test
+    public void testSpatialJoinWithConsiderBoundaryIntersection() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.analyze();
+        spatialRDD.spatialPartitioning(GridType.KDBTREE, 10);
+        SpatialRDD<Geometry> queryRDD = new SpatialRDD<>();
+        queryRDD.rawSpatialRDD = queryDataRdd;
+        queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
+        if (spatialPredicate == SpatialPredicate.INTERSECTS || spatialPredicate == SpatialPredicate.COVERED_BY) {
+            boolean considerBoundaryIntersection = spatialPredicate == SpatialPredicate.INTERSECTS;
+            {
+                JavaPairRDD<Geometry, List<Geometry>> actualResultRdd = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, considerBoundaryIntersection);
+                verifySpatialJoinResult(actualResultRdd);
+            }
+            {
+                JavaPairRDD<Geometry, Geometry> actualResultRdd = JoinQuery.SpatialJoinQueryFlat(spatialRDD, queryRDD, false, considerBoundaryIntersection);
+                verifySpatialJoinFlatResult(actualResultRdd);
+            }
+            {
+                JavaPairRDD<Geometry, Long> actualResultRdd = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, false, considerBoundaryIntersection);
+                verifySpatialJoinCountByKeyResult(actualResultRdd);
+            }
+        }
+    }
+
+    private void verifySpatialJoinResult(JavaPairRDD<Geometry, List<Geometry>> actualResultRdd) {
+        Map<Integer, List<Integer>> actualResults = spatialJoinResultRddToIdMap(actualResultRdd);
+        Assert.assertEquals("Actual result of spatial join should match expected results", expectedResults, actualResults);
+    }
+
+    private void verifySpatialJoinFlatResult(JavaPairRDD<Geometry, Geometry> actualResultRdd) {
+        Map<Integer, List<Integer>> actualResults = spatialJoinFlatResultRddToIdMap(actualResultRdd);
+        Assert.assertEquals("Actual result of spatial join should match expected results", expectedResults, actualResults);
+    }
+
+    private void verifySpatialJoinCountByKeyResult(JavaPairRDD<Geometry, Long> actualResultRdd) {
+        Map<Integer, Long> actualResults = new HashMap<>();
+        actualResultRdd.collect().forEach(pair -> {
+            Geometry queryWindow = pair._1;
+            Long count = pair._2;
+            int queryWindowId = ((UserData) queryWindow.getUserData()).getId();
+            actualResults.put(queryWindowId, count);
+        });
+        Map<Integer, Long> expectedResultsCountByKey = expectedResults.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> (long) e.getValue().size()));
+        Assert.assertEquals("Actual result of spatial join should match expected results", expectedResultsCountByKey, actualResults);
+    }
+
+    private void verifySpatialJoinResult(JavaPairRDD<Geometry, List<Geometry>> expectedResultRdd, JavaPairRDD<Geometry, List<Geometry>> actualResultRdd) {
+        Map<Integer, List<Integer>> expected = spatialJoinResultRddToIdMap(expectedResultRdd);
+        Map<Integer, List<Integer>> actual = spatialJoinResultRddToIdMap(actualResultRdd);
+        Assert.assertEquals("Actual result of distance join should match expected results", expected, actual);
+    }
+
+    private void verifySpatialJoinFlatResult(JavaPairRDD<Geometry, Geometry> expectedResultRdd, JavaPairRDD<Geometry, Geometry> actualResultRdd) {
+        Map<Integer, List<Integer>> expected = spatialJoinFlatResultRddToIdMap(expectedResultRdd);
+        Map<Integer, List<Integer>> actual = spatialJoinFlatResultRddToIdMap(actualResultRdd);
+        Assert.assertEquals("Actual result of distance join should match expected results", expected, actual);
+    }
+
+    private void verifySpatialJoinCountByKeyResult(JavaPairRDD<Geometry, Long> expectedResultRdd, JavaPairRDD<Geometry, Long> actualResultRdd) {
+        Assert.assertEquals("Actual result of distance join should match expected results", expectedResultRdd.collectAsMap(), actualResultRdd.collectAsMap());
+    }
+}
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java
new file mode 100644
index 00000000..cf854061
--- /dev/null
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/RangeQueryTest.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sedona.core.spatialOperator;
+
+import org.apache.sedona.core.enums.IndexType;
+import org.apache.sedona.core.spatialRDD.SpatialRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@RunWith(Parameterized.class)
+public class RangeQueryTest extends SpatialQueryTestBase {
+
+    @BeforeClass
+    public static void onceExecutedBeforeAll() throws IOException {
+        initialize(RangeQueryTest.class.getSimpleName(), "spatial-predicates-test-data.tsv");
+    }
+
+    @AfterClass
+    public static void teardown() {
+        sc.stop();
+    }
+
+    @Parameterized.Parameters(name = "RangeQueryTest-{index}: {0}")
+    public static SpatialPredicate[] spatialPredicates() {
+        return SpatialPredicate.values();
+    }
+
+    private final SpatialPredicate spatialPredicate;
+    private final Geometry queryWindow;
+    private final List<Integer> expectedResults;
+
+    public RangeQueryTest(SpatialPredicate predicate) throws ParseException {
+        this.spatialPredicate = predicate;
+        queryWindow = queryWindowOf(predicate);
+        expectedResults = buildExpectedResults(spatialPredicate, queryWindow);
+        Assert.assertFalse("expected results should not be empty", expectedResults.isEmpty());
+    }
+
+    private static List<Integer> buildExpectedResults(SpatialPredicate spatialPredicate, Geometry queryWindow) {
+        return testDataset.entrySet().stream()
+                .filter(entry -> evaluateSpatialPredicate(spatialPredicate, entry.getValue(), queryWindow))
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toList());
+    }
+
+    private static Geometry queryWindowOf(SpatialPredicate spatialPredicate) throws ParseException {
+        WKTReader wktReader = new WKTReader();
+        switch (spatialPredicate) {
+            case INTERSECTS:
+            case WITHIN:
+            case COVERED_BY:
+            case CROSSES:
+            case OVERLAPS:
+            case TOUCHES:
+                return wktReader.read("POLYGON ((-10 -10, 10 -10, 10 10, -10 10, -10 -10))");
+            case CONTAINS:
+            case COVERS:
+                return wktReader.read("POINT (10 10)");
+            case EQUALS:
+                return wktReader.read("POLYGON ((0 10, 1 10, 1 11, 0 11, 0 10))");
+            default:
+                throw new IllegalArgumentException("Unsupported spatial predicate: " + spatialPredicate);
+        }
+    }
+
+    @Test
+    public void testRangeQueryRaw() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        JavaRDD<Geometry> queryResultRdd = RangeQuery.SpatialRangeQuery(spatialRDD, queryWindow, spatialPredicate, false);
+        verifyQueryResult(queryResultRdd);
+    }
+
+    @Test
+    public void testRangeQueryWithIndex() throws Exception {
+        SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+        spatialRDD.rawSpatialRDD = inputRdd;
+        spatialRDD.buildIndex(IndexType.RTREE, false);
+        JavaRDD<Geometry> queryResultRdd = RangeQuery.SpatialRangeQuery(spatialRDD, queryWindow, spatialPredicate, true);
+        verifyQueryResult(queryResultRdd);
+    }
+
+    @Test
+    public void testRangeQueryWithConsiderBoundaryIntersection() throws Exception {
+        if (spatialPredicate == SpatialPredicate.INTERSECTS || spatialPredicate == SpatialPredicate.COVERED_BY) {
+            boolean considerBoundaryIntersection = spatialPredicate == SpatialPredicate.INTERSECTS;
+            SpatialRDD<Geometry> spatialRDD = new SpatialRDD<>();
+            spatialRDD.rawSpatialRDD = inputRdd;
+            JavaRDD<Geometry> queryResultRdd = RangeQuery.SpatialRangeQuery(spatialRDD, queryWindow, considerBoundaryIntersection, false);
+            verifyQueryResult(queryResultRdd);
+        }
+    }
+
+    private void verifyQueryResult(JavaRDD<Geometry> resultRdd) {
+        List<Integer> actualResults = resultRdd.map(geom -> ((UserData) geom.getUserData()).getId()).collect();
+        Assert.assertEquals("Number of results should match with expected results", expectedResults.size(), actualResults.size());
+        List<Integer> sortedActualResults = actualResults.stream().sorted(Integer::compareTo).collect(Collectors.toList());
+        List<Integer> sortedExpectedResults = expectedResults.stream().sorted(Integer::compareTo).collect(Collectors.toList());
+        Assert.assertEquals("Actual range query result should match with expected results", sortedExpectedResults, sortedActualResults);
+    }
+}
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/SpatialQueryTestBase.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/SpatialQueryTestBase.java
new file mode 100644
index 00000000..99e5ef73
--- /dev/null
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/SpatialQueryTestBase.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.sedona.core.spatialOperator;
+
+import org.apache.sedona.core.TestBase;
+import org.apache.spark.api.java.JavaRDD;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.io.ParseException;
+import org.locationtech.jts.io.WKTReader;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.stream.Stream;
+
+public class SpatialQueryTestBase extends TestBase {
+    protected static JavaRDD<Geometry> inputRdd;
+    protected static Map<Integer, Geometry> testDataset;
+
+    protected static void initialize(String testName, String testDataPath) throws IOException {
+        initialize(testName);
+        sc.setLogLevel("ERROR");
+        inputRdd = readTestDataAsRDD(testDataPath);
+        testDataset = readTestDataAsMap(testDataPath);
+    }
+
+    private static Geometry lineToGeometry(String line) {
+        String[] lineSplit = line.split("\t");
+        WKTReader reader = new WKTReader();
+        try {
+            Geometry geom = reader.read(lineSplit[2]);
+            geom.setUserData(new RangeQueryTest.UserData(Integer.parseInt(lineSplit[0]), lineSplit[1]));
+            return geom;
+        } catch (ParseException e) {
+            throw new RuntimeException("Invalid geometry data in line: " + line, e);
+        }
+    }
+
+    protected static JavaRDD<Geometry> readTestDataAsRDD(String fileName) {
+        String inputLocation = RangeQueryTest.class.getClassLoader().getResource(fileName).getPath();
+        return sc.textFile("file://" + inputLocation).map(SpatialQueryTestBase::lineToGeometry);
+    }
+
+    protected static Map<Integer, Geometry> readTestDataAsMap(String fileName) throws IOException {
+        String inputLocation = RangeQueryTest.class.getClassLoader().getResource(fileName).getPath();
+        try (Stream<String> lines = Files.lines(Paths.get(inputLocation))) {
+            Map<Integer, Geometry> map = new HashMap<>();
+            lines.forEach(line -> {
+                Geometry geom = lineToGeometry(line);
+                int id = ((RangeQueryTest.UserData) geom.getUserData()).getId();
+                map.put(id, geom);
+            });
+            return map;
+        }
+    }
+
+    protected static boolean evaluateSpatialPredicate(SpatialPredicate spatialPredicate, Geometry geom1, Geometry geom2) {
+        switch (spatialPredicate) {
+            case INTERSECTS:
+                return geom1.intersects(geom2);
+            case CONTAINS:
+                return geom1.contains(geom2);
+            case WITHIN:
+                return geom1.within(geom2);
+            case COVERS:
+                return geom1.covers(geom2);
+            case COVERED_BY:
+                return geom1.coveredBy(geom2);
+            case OVERLAPS:
+                return geom1.overlaps(geom2);
+            case CROSSES:
+                return geom1.crosses(geom2);
+            case EQUALS:
+                return geom1.equals(geom2);
+            case TOUCHES:
+                return geom1.touches(geom2);
+            default:
+                throw new IllegalArgumentException("Unknown spatial predicate: " + spatialPredicate);
+        }
+    }
+
+    protected static class UserData {
+        private final int id;
+        private final String name;
+
+        public UserData(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+
+        public int getId() {
+            return id;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        @Override
+        public int hashCode() {
+            return id;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this) {
+                return true;
+            }
+            if (!(obj instanceof RangeQueryTest.UserData)) {
+                return false;
+            }
+            RangeQueryTest.UserData other = (RangeQueryTest.UserData) obj;
+            return id == other.id && name.equals(other.name);
+        }
+    }
+}
diff --git a/core/src/test/resources/spatial-join-query-window.tsv b/core/src/test/resources/spatial-join-query-window.tsv
new file mode 100644
index 00000000..856fb95a
--- /dev/null
+++ b/core/src/test/resources/spatial-join-query-window.tsv
@@ -0,0 +1,207 @@
+0	point_0	POINT (4.76679951997882 4.475190399610504)
+1	poly_0	POLYGON ((4.798946127648378 -2.597379083462104, 3.998605082952126 -2.5871635414241876, 1.3619844390484364 -2.4700322379621498, 1.5766957531101233 1.0919883443822598, 4.172575283769907 1.0847768340086617, 4.798946127648378 -2.597379083462104))
+2	point_1	POINT (6.3250795931592245 -0.9212866163718179)
+3	poly_1	POLYGON ((2.036242003613342 -2.914869452474756, 1.6305681582754468 -2.362737456455594, 1.7177341916657025 0.6631162661118815, 2.37654838009537 0.4968834995343963, 5.2248969613129255 -2.553318005476249, 5.137330677033438 -2.794555699042704, 2.036242003613342 -2.914869452474756))
+4	point_2	POINT (-3.9668389205164067 0.1355762398647804)
+5	poly_2	POLYGON ((8.699761743542366 2.9090906742971567, 5.642529176137788 2.95611990291598, 4.900823695917716 3.1152168002728384, 5.074571003368659 5.545378012823112, 5.0960650916278905 5.664370415944729, 7.903245505857866 5.696504311790811, 8.699761743542366 2.9090906742971567))
+6	point_3	POINT (10.525156126022221 -7.2954942432352325)
+7	poly_3	POLYGON ((9.223300025388987 -1.0515735815680187, 5.4191495311353695 -0.7871356624133572, 5.8448878038565475 2.584493678127774, 8.93394788011522 2.5352877942019667, 9.223300025388987 -1.0515735815680187))
+8	point_4	POINT (2.3246318554307366 2.1451536807847265)
+9	poly_4	POLYGON ((8.550714679481104 6.840321825808029, 5.137868914152924 7.32040724406054, 5.752935135165679 10.515955864584809, 7.850956280939767 10.410013835539585, 8.373816248199118 9.680288966842724, 8.550714679481104 6.840321825808029))
+10	point_5	POINT (2.879917116970978 -8.051531454541987)
+11	poly_5	POLYGON ((7.57545549207161 4.669743948764811, 5.353437513645368 5.16400190266681, 4.802161127467281 7.817052515242055, 5.043342770782431 8.040365763730444, 8.074334271641572 7.833002044355053, 8.27597511667081 5.3868634908314466, 7.57545549207161 4.669743948764811))
+12	point_6	POINT (-2.244365867255443 0.2463342543389093)
+13	poly_6	POLYGON ((-2.2830519079945786 5.422978590520493, -2.6157736163550354 5.459473684245674, -2.974578108333854 5.533631285356313, 0.622659152043904 9.177475197461924, 0.8733505296537327 5.9086029331421415, -2.2830519079945786 5.422978590520493))
+14	point_7	POINT (-4.024597345979839 11.018483614916434)
+15	poly_7	POLYGON ((4.682592485844874 -2.3749482412065483, 4.99513842903404 1.3134761387558376, 5.142508238193179 1.4690047139299904, 7.874253761922389 1.062568761454282, 8.445809562305978 -1.6692072411661094, 4.682592485844874 -2.3749482412065483))
+16	point_8	POINT (-2.465746053205309 -7.2613655910436705)
+17	poly_8	POLYGON ((-1.2894700829741912 -11.174248820629527, -4.888105186856261 -10.972302318874904, -4.093568868228279 -7.589379855330513, -1.9369110208233544 -7.583757195129115, -1.8364393734043536 -7.629047627082136, -1.0629940193726213 -10.59433756932782, -1.2894700829741912 -11.174248820629527))
+18	point_9	POINT (5.303239347538817 6.57874649487068)
+19	poly_9	POLYGON ((2.7913998610977324 3.488966415491594, 3.188054883684913 6.1595509962654065, 6.3625699778073805 6.509256850009678, 6.730314357016587 6.216460529771824, 6.555345715365529 3.692665327213031, 2.7913998610977324 3.488966415491594))
+20	point_10	POINT (-4.2080696027812925 -2.294892757801251)
+21	poly_10	POLYGON ((-3.2311875301803257 -2.8907129519579837, -5.685654789952819 -2.47670257096096, -5.699742834833257 -2.2527012462515934, -5.315514853110718 0.4223756788334789, -3.0142882520822076 0.5248511054665805, -2.397054880699796 0.3862600515054888, -2.9725783181005014 -2.1588519982223597, -3.2311875301803257 -2.8907129519579837))
+22	point_11	POINT (-1.3346543466462146 -9.431888485873547)
+23	poly_11	POLYGON ((-6.26889181589464 -3.0512516520237005, -6.375556588632167 0.2972890740058034, -6.186125268217607 0.6128117752153015, -5.7837011798824625 0.4287030709361921, -2.857493608135134 -2.663596320558162, -3.145225326411492 -2.8422404085757105, -6.26889181589464 -3.0512516520237005))
+24	point_12	POINT (-7.703141319815678 -4.310590934528795)
+25	poly_12	POLYGON ((-7.32596945151867 6.480662886970704, -10.552149426989045 6.513343264828242, -10.586972115362151 7.376646929456772, -7.350975994189993 10.303504637583366, -6.813580560023734 9.963850230807095, -6.9687136229983295 6.927773467494911, -7.32596945151867 6.480662886970704))
+26	point_13	POINT (-5.875883551082511 -3.998350684647485)
+27	poly_13	POLYGON ((1.7744791092650054 -1.1237191574231136, 1.6953537438004218 1.7514712811507516, 1.7114256507498429 2.2069821346767426, 2.4877152395288364 2.5114569696593505, 5.119480327671727 2.4724759410348094, 5.420036697091164 2.186554378630453, 2.390209176249573 -0.8934112040799456, 1.7744791092650054 -1.1237191574231136))
+28	point_14	POINT (-4.145655030518807 -3.4580442670971605)
+29	poly_14	POLYGON ((-10.434488010518553 3.3379544791535136, -10.982051606147435 6.109324286790611, -7.396486038611167 6.282956602262054, -7.255438018361381 3.504503080725498, -7.829408484461852 3.4417039350546994, -10.434488010518553 3.3379544791535136))
+30	point_15	POINT (-4.445608245737867 -6.387921668054921)
+31	poly_15	POLYGON ((6.059373460537813 -3.58126095052868, 3.5194283846087435 -3.5197168946744224, 2.8382494058614074 -3.1366865694236767, 3.142542836061076 -0.06642182484574, 3.5202858551759357 0.2955540177456246, 5.657223923053264 0.2657423709743907, 5.9665856892492855 -0.0723109292874276, 6.059373460537813 -3.58126095052868))
+32	point_16	POINT (7.122902869971803 -0.9141356220266001)
+33	poly_16	POLYGON ((3.6148640404066965 4.565338333652741, 1.2494079443480435 4.575499828455724, 0.7525311589709689 6.954459461754675, 3.958411403379066 7.697313268978823, 4.170588740542943 7.701700270636262, 3.6148640404066965 4.565338333652741))
+34	point_17	POINT (8.505335997481128 0.0049030482219634)
+35	poly_17	POLYGON ((4.079515751160459 4.137248751959882, 1.6602026846478828 7.9936341251802645, 4.534187018901908 7.755368956001519, 4.674139100375884 7.591614816158574, 4.079515751160459 4.137248751959882))
+36	point_18	POINT (-4.026885998693439 3.4056599393571276)
+37	poly_18	POLYGON ((2.1391877498958243 -3.4209978136350485, 1.9993360144166683 -3.414315960624905, -1.1726307530821054 -1.0880718009730845, -1.534311773046994 -0.6798449922358829, -1.390725648956554 -0.5915596317098546, 2.2345511869732984 -0.7579399198009744, 2.194698853449954 -3.2858899044948355, 2.1391877498958243 -3.4209978136350485))
+38	point_19	POINT (2.4506084233513477 -1.140045397098659)
+39	poly_19	POLYGON ((9.108273983328894 -10.993712654852581, 6.055467389904093 -7.858456223524288, 8.581529451397573 -7.156467681322203, 8.921811765110528 -8.021172158089897, 9.108273983328894 -10.993712654852581))
+40	point_20	POINT (0.6488997717280272 5.9492914593091415)
+41	poly_20	POLYGON ((7.761076856940239 -4.595546609299049, 7.598490852184215 -1.1452165915057178, 8.37401947182239 -1.1059470907921343, 10.491006439089425 -1.690667292237528, 7.761076856940239 -4.595546609299049))
+42	point_21	POINT (2.5823263024706904 6.667474484538573)
+43	poly_21	POLYGON ((1.6486616752527026 -1.1550419697446053, -0.6259771252599724 -0.7492593535394629, -0.9026970475206735 2.4588534138224976, 2.2010595388414655 2.401437064446783, 2.410969453725435 -1.102472059678811, 1.6486616752527026 -1.1550419697446053))
+44	point_22	POINT (10.549509207211772 -10.26107624039254)
+45	poly_22	POLYGON ((7.3267760771226635 -7.13468998469347, 6.669483925325213 -6.763000425365381, 6.779450914838263 -4.106378371665523, 9.932492397645358 -3.1771982409196715, 10.096038176056368 -4.011307569707306, 10.136137265784805 -6.195112670070586, 7.3267760771226635 -7.13468998469347))
+46	point_23	POINT (5.792545600471227 1.0132953404189118)
+47	poly_23	POLYGON ((-5.454397306950558 -1.3280666452130472, -5.838548069765342 2.223179124279732, -2.5702644252021494 2.1944945075477738, -2.097809552011033 -1.110758695813761, -5.454397306950558 -1.3280666452130472))
+48	point_24	POINT (6.730084485380335 1.3675491537011788)
+49	poly_24	POLYGON ((9.898748086151494 -1.776339483020429, 6.637783340489857 -1.0619067688503436, 6.651971196938736 1.3606221406248784, 9.468588724876927 2.0401895233796448, 9.82066010772336 1.5086048847614348, 9.898748086151494 -1.776339483020429))
+50	point_25	POINT (-5.210334302621697 7.67426828942861)
+51	poly_25	POLYGON ((-1.6391127686384672 7.935935005065685, -1.7903056889299842 11.305999869289556, -1.2013060912824458 10.976408504378513, -1.1178282384003042 8.552712344701614, -1.533973711931627 8.052792797963306, -1.6391127686384672 7.935935005065685))
+52	point_26	POINT (6.534351092926136 3.304050846663311)
+53	poly_26	POLYGON ((2.95773731082753 -2.7297810103572595, 2.895821303627671 0.0013972781298768, 3.057008946923382 0.3983377129076202, 6.121611293637422 0.4750026322443834, 6.47537087816324 -0.0731909031241327, 2.95773731082753 -2.7297810103572595))
+54	point_27	POINT (1.3575172933861348 -10.89419033733823)
+55	poly_27	POLYGON ((-9.35955520874825 -7.314439216222125, -9.544045806113237 -6.676683148755391, -9.423722736835348 -4.049461718750809, -5.837247734502015 -3.5501114980293957, -6.298547257729137 -6.607482693310921, -6.615008671498528 -7.247202829499537, -9.35955520874825 -7.314439216222125))
+56	point_28	POINT (-1.4717614909936723 -0.8669303210778863)
+57	poly_28	POLYGON ((5.586849349181474 6.2753799357639775, 5.486838528018766 10.13667283521102, 9.026099151733607 10.008215952525225, 9.041284739232902 6.4805020013925425, 8.731925697137914 6.4186721203636115, 5.586849349181474 6.2753799357639775))
+58	point_29	POINT (-10.536442979187465 -7.755789163331447)
+59	poly_29	POLYGON ((8.307588223201726 -8.600673401980485, 8.449487943411818 -5.20898603194653, 11.610392502339922 -4.994413401674748, 11.825546671041058 -5.523890833983221, 11.505783413763833 -8.38779012365573, 8.307588223201726 -8.600673401980485))
+60	point_30	POINT (-0.6768662274979498 -5.315935013401525)
+61	poly_30	POLYGON ((-7.563110130051724 -9.7619102116246, -10.411421906497246 -6.489598493301241, -10.346346023272556 -6.0291062626029355, -10.24983700016513 -5.94162214535509, -7.289961273102721 -6.161243981192226, -7.157961745422749 -8.864724394669558, -7.563110130051724 -9.7619102116246))
+62	point_31	POINT (-6.784089651022844 -7.582435358399982)
+63	poly_31	POLYGON ((3.649789123566843 -7.8518282767760805, -0.1952202600495916 -7.421569930613016, -0.2766852531117388 -4.435638066558665, 3.4817337354655553 -4.076741317528576, 3.649789123566843 -7.8518282767760805))
+64	point_32	POINT (2.624550035859797 5.893383085295463)
+65	poly_32	POLYGON ((-0.8535879275090981 7.872783515456421, -3.634555338129799 7.910713778606024, -4.177117548783257 10.668858895384293, -0.5769724071443503 11.122142501200358, -0.6629705572201541 8.50182834518073, -0.8535879275090981 7.872783515456421))
+66	point_33	POINT (-10.17370704900322 6.204552530497558)
+67	poly_33	POLYGON ((-11.23576420944176 2.125686346814396, -11.340565774206716 2.702973336269335, -11.218319857594278 5.724010762312261, -7.98327959628247 5.0690030756970055, -7.938911044272372 2.2200120519080735, -11.23576420944176 2.125686346814396))
+68	point_34	POINT (-0.9116775837903837 7.717468461989634)
+69	poly_34	POLYGON ((-1.5553530465672263 -3.4158086348122074, -1.6183661529793498 -0.049731198591755, -1.3418257503604312 0.2626795078513688, 1.4385725972580592 -0.0076445495278534, 1.607005418796618 -2.9838237909152445, -1.5553530465672263 -3.4158086348122074))
+70	point_35	POINT (10.22609856832701 2.2294767576965007)
+71	poly_35	POLYGON ((9.739438702809297 -2.340939081738172, 6.762898698192063 -2.2980393428619585, 6.248389363842631 -2.2741100537026036, 6.249016970976516 -1.4850548053848949, 6.32499155076229 1.4210635317296785, 10.025787564905267 1.2253936156108822, 9.739438702809297 -2.340939081738172))
+72	point_36	POINT (-7.3656428390524304 4.005119440524868)
+73	poly_36	POLYGON ((0.7056584971995963 -6.596504988895643, -2.4501337523508284 -6.286404747933218, -2.2291254198549835 -5.981462337926054, 0.758700697460597 -5.638854162784499, 0.7056584971995963 -6.596504988895643))
+74	point_37	POINT (5.876116021304367 -7.35667006648831)
+75	poly_37	POLYGON ((5.2610506668076695 -2.3273529853477766, 5.407005765891191 0.8227624310768391, 5.58606366116848 1.009815771804118, 5.975253971008129 1.3471299780257018, 8.427509797924003 1.0552274023083141, 8.644827369747366 0.910002632552052, 8.185192798816647 -1.7087879855070822, 5.2610506668076695 -2.3273529853477766))
+76	point_38	POINT (5.291890957841855 -3.4978846986164966)
+77	poly_38	POLYGON ((5.460587694116024 -4.298621053368699, 1.790795587267861 -3.7465187175109422, 1.7643076991781266 -1.5178098954700245, 2.081568126088605 -1.3394653357476938, 5.1147425589930595 -0.7008890123734983, 5.460587694116024 -4.298621053368699))
+78	point_39	POINT (8.078340046561136 -6.564637897186193)
+79	poly_39	POLYGON ((3.9840600086267943 -11.30737091974522, 3.7917554656762382 -11.27749408148169, 6.061171933223869 -8.206371665072492, 6.205988557126835 -8.018600828939242, 6.301045516374494 -8.297743129699425, 6.972942619378503 -10.779496198343239, 6.99795240153337 -10.977992149095837, 6.624166117206439 -11.054335946073804, 3.9840600086267943 -11.30737091974522))
+80	point_40	POINT (-8.197510051246786 -4.151001901741971)
+81	poly_40	POLYGON ((8.52321094069324 6.5950787088011875, 4.684825352256645 6.66862049344423, 4.790512864806389 8.772091217291667, 4.865432130230244 9.402023785744944, 7.657182628318176 9.681711956385012, 8.464610172913149 9.513776882975199, 8.52321094069324 6.5950787088011875))
+82	point_41	POINT (-5.198772048577315 -4.877869649272661)
+83	poly_41	POLYGON ((4.71865020296401 -5.492834525799387, 4.858472644636721 -2.8405592337270127, 5.254416836378171 -2.573753899619125, 8.123901735105443 -2.096263794332922, 8.470790838522026 -4.8902081349871, 4.71865020296401 -5.492834525799387))
+84	point_42	POINT (4.3286475132522515 -5.788019855141347)
+85	poly_42	POLYGON ((-7.6685411846504286 -8.891153041862953, -11.080976738896076 -8.785993237649748, -10.848704435729092 -5.816592800027336, -10.488901040497293 -5.313187174363407, -7.2570610638078765 -8.451430013847883, -7.6685411846504286 -8.891153041862953))
+86	point_43	POINT (-1.686348109071523 7.077790875960081)
+87	poly_43	POLYGON ((-6.318936372442209 -6.44985859539768, -8.669092633645995 -3.0659222341103956, -6.264600073171498 -3.075347218794894, -5.3654318906014495 -3.1019726170919877, -5.488002156793005 -5.892626167859213, -6.318936372442209 -6.44985859539768))
+88	point_44	POINT (2.1083522906844867 1.520611745331183)
+89	poly_44	POLYGON ((-6.488221091101698 -6.822189581074655, -10.108438538425933 -6.729762024510054, -7.2648085240474565 -3.2308424662415165, -6.488221091101698 -6.822189581074655))
+90	point_45	POINT (-3.5951961593038044 5.271981711057167)
+91	poly_45	POLYGON ((6.991982220952351 -4.840300579087192, 4.674000760185384 -4.586057393971918, 4.388434404041147 -4.2943311376499445, 7.158116421745062 -1.23457853575738, 7.607875131184942 -1.0602283300774458, 7.652615813666654 -1.9125724580336896, 7.710474655900256 -4.826140025478216, 6.991982220952351 -4.840300579087192))
+92	point_46	POINT (5.549527374259334 1.7955605845507532)
+93	poly_46	POLYGON ((-1.5335328635008265 7.002636133543222, -5.00414378873779 10.07838283786275, -4.926371629790907 10.256844193980395, -4.189217028579433 10.354187512094835, -2.109889928168611 9.823107561400544, -1.1894595031141169 7.369711153876088, -1.5335328635008265 7.002636133543222))
+94	point_47	POINT (8.837205261086202 2.620424226501472)
+95	poly_47	POLYGON ((-7.3116881410656696 3.473288309912551, -9.773341256862274 4.0335805870185535, -10.671842204840717 6.401194671934064, -9.994531775467811 6.617747904825614, -7.197225134433781 7.228645610013984, -6.773636247826083 3.589175421287749, -7.3116881410656696 3.473288309912551))
+96	point_48	POINT (-6.917336413661786 3.969229546467444)
+97	poly_48	POLYGON ((1.6257512168790866 0.667624520573268, 1.0285668733623496 0.7829808524070376, -1.6716889409121616 3.1069142663409677, -1.287883437510213 3.831220786866907, 1.6995071149933305 3.899448011461474, 1.6257512168790866 0.667624520573268))
+98	point_49	POINT (1.2716240545958621 -7.228019081862617)
+99	poly_49	POLYGON ((-5.570762359283394 -9.302487838634423, -5.590262585411654 -6.3457033816192805, -2.2954931188335364 -5.812159446688176, -2.0390130430589086 -5.801902710263647, -1.6439318501976645 -5.79125619055238, -2.411219083172183 -9.03451505291961, -5.570762359283394 -9.302487838634423))
+100	point_50	POINT (-4.150386199125563 -9.454803987923851)
+101	poly_50	POLYGON ((-3.20049559721474 -4.553247621148784, -2.711623263236095 -0.8989783720810074, 0.3817620991967066 -1.1032569342495055, 0.5403422266863553 -4.322661336554819, -3.20049559721474 -4.553247621148784))
+102	point_51	POINT (-8.509464319942971 3.087163559102354)
+103	poly_51	POLYGON ((8.939932205532987 -1.4894337398231001, 6.171690391819336 -0.9372582007233352, 6.078099234773756 1.5961770095845675, 9.127013330853234 1.4503668412294908, 9.300257545582848 1.426311722654333, 9.135887786417154 -1.4243888585278204, 8.939932205532987 -1.4894337398231001))
+104	point_52	POINT (2.2967394503428324 2.420794319215304)
+105	poly_52	POLYGON ((8.232236921401695 1.2790248123052137, 5.579472517346639 1.3298603510321663, 5.771294842007195 4.400947591082607, 8.270161589429197 5.151048018881543, 9.042361370299862 4.981970071404799, 9.048532910747355 4.8802245279633905, 8.232236921401695 1.2790248123052137))
+106	point_53	POINT (1.1014185160389658 7.7054893558066375)
+107	poly_53	POLYGON ((2.0632929087183576 5.776764331313336, -0.9139916734074951 5.819998892324443, -1.2843981659290935 8.880387928666426, -0.5186949317791023 8.804754413099143, 2.2178283199334996 8.37185556130272, 2.0632929087183576 5.776764331313336))
+108	point_54	POINT (4.900018914282487 -1.2441662785593932)
+109	poly_54	POLYGON ((-2.231536148598427 4.862755107246176, -5.670511949049979 5.135886507203857, -4.91826998499967 8.482640336370572, -2.065377480628678 8.55519256769365, -1.8239913869860036 8.086883781889279, -1.6964603008398078 7.615858402471165, -2.231536148598427 4.862755107246176))
+110	point_55	POINT (7.190034092976892 8.101161688050524)
+111	poly_55	POLYGON ((0.8086407565518523 -3.61058400127611, -1.9694097511327588 -0.0589746606714905, -1.5670278562115536 0.1011526557799705, 0.602715231394739 0.155951145638273, 1.2515536629575577 -0.6571839858902511, 0.8086407565518523 -3.61058400127611))
+112	point_56	POINT (4.170509163980153 -0.7349628740477612)
+113	poly_56	POLYGON ((-8.701901590587656 1.4045047505455721, -9.07609759089704 1.8480905019215998, -9.513583445384272 4.679620313165456, -6.012288635893112 5.13010133934686, -5.739912351305906 4.605767997318195, -8.701901590587656 1.4045047505455721))
+114	point_57	POINT (-11.076110999799758 -4.63613260666452)
+115	poly_57	POLYGON ((11.44455620173429 2.41146569183704, 8.474827062455212 2.4595570358978986, 8.506983266109403 2.878105937590976, 11.138945899465648 5.814112371296289, 11.314997732039878 5.844973677128826, 11.44455620173429 2.41146569183704))
+116	point_58	POINT (-8.895844926175085 -9.404982916071459)
+117	poly_58	POLYGON ((-9.890619055287802 2.6714241990236123, -9.369398244124357 5.931980840822687, -6.368604784188088 6.072126964116887, -6.242026207122515 5.909468255863784, -6.5802778056763405 3.403891123866895, -9.890619055287802 2.6714241990236123))
+118	point_59	POINT (-4.3980586561479065 8.101801937932995)
+119	poly_59	POLYGON ((2.7996882305136417 -0.9742040607383546, 2.4172818976354273 1.4839108885386558, 5.987856731247096 1.8512948600618078, 5.637894871643754 -0.963190743722699, 5.4386198815194895 -0.9729047666811583, 2.7996882305136417 -0.9742040607383546))
+120	point_60	POINT (8.295101607055434 -2.537571597141872)
+121	poly_60	POLYGON ((-6.088058741625141 -9.606472253755049, -6.462932342936586 -6.217292649223163, -6.240451788173231 -5.817518843747319, -2.993568598561392 -5.894630825177279, -3.8601455625649415 -9.351759266319736, -6.088058741625141 -9.606472253755049))
+122	point_61	POINT (3.4542084746016952 -6.928033369735417)
+123	poly_61	POLYGON ((8.114690229917272 8.366935506843179, 8.181500943300083 11.448482221830293, 8.604761432880291 11.756052630314164, 9.007231188254808 8.594671527874759, 8.114690229917272 8.366935506843179))
+124	point_62	POINT (-8.01275505255707 6.152920561968614)
+125	poly_62	POLYGON ((2.3832045282731578 -1.2358623973107574, -1.1661721997155476 -0.927910054427161, -0.7069437473494451 2.4636133613775506, 2.3107433230143037 2.6398067905379294, 2.6247448215863867 -0.8823015288016753, 2.3832045282731578 -1.2358623973107574))
+126	point_63	POINT (7.958915447981197 -3.586602207437128)
+127	poly_63	POLYGON ((-6.178367981968066 6.681551333278489, -6.802306379859333 10.206604308390267, -3.225682295281873 10.18858980715834, -3.4489435283404415 6.743283468258898, -6.178367981968066 6.681551333278489))
+128	point_64	POINT (-3.888075839757021 8.840514407109769)
+129	poly_64	POLYGON ((3.928859680515985 -7.081058605896757, 6.110753931409558 -3.368255842734081, 6.430373080580987 -3.8315660559700504, 6.8340393385346125 -6.587248479500316, 6.778411289694773 -7.011224364738455, 3.928859680515985 -7.081058605896757))
+130	point_65	POINT (2.5310090159710885 -9.481229329274178)
+131	poly_65	POLYGON ((-7.791394738337886 6.550654787128517, -10.603757611170106 6.617831164268308, -11.066641546834017 9.722256432562189, -11.062584757920947 9.806126596353312, -7.603756616321385 10.379422634628213, -7.230429473989082 10.088996079666058, -7.1413047490296915 6.642110796911304, -7.791394738337886 6.550654787128517))
+132	point_66	POINT (0.1822686353330205 -1.4597374519752893)
+133	poly_66	POLYGON ((-2.5099170715177372 5.775262506543776, -6.362202512727217 8.803479350596314, -6.31283549934688 8.998720002534988, -3.0113153433743403 8.711760461262184, -2.5099170715177372 5.775262506543776))
+134	point_67	POINT (2.1417767592585673 2.6816156770670494)
+135	poly_67	POLYGON ((1.4716004907242415 4.708704442925756, -1.1969373305808086 5.1424217067116595, -1.5863654336405495 7.898143478852565, -1.2286815327145373 8.23461618756997, 1.119878086284824 8.355000298886608, 1.5473511052328295 4.795578627117424, 1.4716004907242415 4.708704442925756))
+136	point_68	POINT (2.3992693265505327 -10.025875394254596)
+137	poly_68	POLYGON ((1.5369019058163058 -8.560720230818788, -0.9233702292027484 -8.557635562790084, -1.5380582316878304 -5.702258224526114, 1.6702725758000279 -5.235783808528586, 2.2384266301441373 -5.474240193572563, 1.5369019058163058 -8.560720230818788))
+138	point_69	POINT (3.1080448783578447 -2.821071224782648)
+139	poly_69	POLYGON ((0.1598412411487806 1.9408754682027518, -0.0316948245424238 2.8392126480578552, -0.1945509738028299 5.744718988731156, 2.9073704822167272 5.798540964534614, 3.594276260228508 5.785697026476725, 0.1598412411487806 1.9408754682027518))
+140	point_70	POINT (-0.7305433839600497 -4.750044215258399)
+141	poly_70	POLYGON ((-7.01202494554884 3.479592698011067, -7.389880548231477 3.717065427406071, -7.07160846870839 6.548931925282833, -6.8556505361716615 7.226343345576453, -4.741573185660447 7.267648200643926, -3.9795659975992757 3.9599195727872365, -7.01202494554884 3.479592698011067))
+142	point_71	POINT (1.8122319570335157 -6.149275042346195)
+143	poly_71	POLYGON ((-9.489260476841896 -10.94345382383466, -10.220154359659077 -10.396163156327042, -9.886128091719453 -7.75700822725714, -6.611759266246247 -7.7732768058432775, -9.489260476841896 -10.94345382383466))
+144	point_72	POINT (9.363800412843547 5.586319418191174)
+145	poly_72	POLYGON ((-6.23358949281625 -11.209532954352758, -9.849968653586181 -11.123432982981917, -9.688364085829637 -7.8629681067415635, -9.406879749435806 -7.559193760010661, -6.463709071140336 -7.590267553472225, -6.219888882087733 -11.047232129837374, -6.23358949281625 -11.209532954352758))
+146	point_73	POINT (10.521509109439023 10.341561370895118)
+147	poly_73	POLYGON ((1.1711254556906794 4.40082815631202, -1.44223250053511 4.967359561077581, -1.7321788389005617 7.97846464962601, 1.7921754798612453 8.185852689489884, 1.637625126034789 4.953395541012718, 1.1711254556906794 4.40082815631202))
+148	point_74	POINT (2.8873741789716068 6.302835286778058)
+149	poly_74	POLYGON ((-3.5287308972164277 -4.435084102251704, -3.785248067048451 -0.6384894331714412, -0.7441251070080899 -1.1388076903962023, -0.1546337047739019 -3.699735546888806, -0.3508338546957792 -3.806109808023958, -3.5287308972164277 -4.435084102251704))
+150	point_75	POINT (-10.461407509001326 -9.139709998599727)
+151	poly_75	POLYGON ((2.806380917515354 4.121483150916959, -0.126668325584244 7.17136601338164, -0.0213920542307151 8.008746869850581, 2.5159526014463833 7.82183803515479, 2.6205530069641094 7.801321359696121, 2.89710049951473 4.966380677449424, 2.806380917515354 4.121483150916959))
+152	point_76	POINT (-5.359067901094271 1.4145160924384248)
+153	poly_76	POLYGON ((-4.579998681295177 -3.4883130682697585, -7.930935663482661 -3.2572218569000464, -7.499147020987206 -0.2314993902415563, -4.918087389948708 0.1261545511992539, -4.5102110806673705 -0.4745934528810367, -4.579998681295177 -3.4883130682697585))
+154	point_77	POINT (-6.726331610316842 3.7096842301688753)
+155	poly_77	POLYGON ((-6.539815448612083 -8.121583305873484, -6.882269224740196 -7.9542420503994204, -7.188626045910372 -5.255327720231394, -6.859239862482178 -4.909040697293072, -4.077752877628329 -4.427497828615437, -4.049139882185548 -4.64314162234276, -4.232445992229448 -8.067026403800613, -6.539815448612083 -8.121583305873484))
+156	point_78	POINT (-8.12471462737865 3.147967058788076)
+157	poly_78	POLYGON ((5.2162299230796725 7.586470234471373, 2.8209120578114026 7.873176539747663, 2.4514504382018814 10.62234566360787, 6.14036238001332 11.191384586808311, 5.859619485535134 8.128739183469946, 5.2162299230796725 7.586470234471373))
+158	point_79	POINT (6.797312797871147 6.351501042632971)
+159	poly_79	POLYGON ((6.466148530332612 1.4652245186582222, 3.7532352689143593 1.6054765321506288, 3.164724745854542 2.211137091991092, 3.212830120996944 2.3499738175968687, 6.793049822535464 4.944962269748611, 6.990019343878642 2.4016185046785354, 6.466148530332612 1.4652245186582222))
+160	point_80	POINT (-7.417002229669077 -5.486641150186435)
+161	poly_80	POLYGON ((2.2129912205457214 -11.891331778406375, 2.3507957413195024 -8.590622223736153, 5.658592433722509 -8.145766750130528, 5.815138705985 -8.908100903418863, 5.818036550450943 -11.327742589661709, 2.2129912205457214 -11.891331778406375))
+162	point_81	POINT (-5.377980931875971 5.357196519885756)
+163	poly_81	POLYGON ((0.7133374678505431 -0.0523912222856158, -2.9387202640584196 0.0381991759352069, -3.0713033144361708 0.7167030979434694, -2.849581827832455 3.421852954676239, 0.1007465026138927 3.413751113547451, 0.7133374678505431 -0.0523912222856158))
+164	point_82	POINT (-5.629586398865214 -2.1935693267592953)
+165	poly_82	POLYGON ((2.642014898522996 2.8328008155026407, -0.3304772863090388 6.116862059056643, 3.222375843942155 6.486986883975444, 3.282511090211747 6.141005827800766, 3.2049907347386006 3.39934592740251, 2.642014898522996 2.8328008155026407))
+166	point_83	POINT (-5.573658564847905 -9.475932631533697)
+167	poly_83	POLYGON ((5.987183879290608 5.035891752823212, 5.6509245544362114 5.200951876202498, 2.5474801063806374 8.309273933279664, 5.965313385611381 8.571831388249409, 6.2767123311552675 5.585983339485294, 6.234250627771025 5.4878627031934, 5.987183879290608 5.035891752823212))
+168	point_84	POINT (5.142933998466797 1.3459114519973943)
+169	poly_84	POLYGON ((-9.836853359751249 7.100441764640771, -10.121028370635333 7.41813657368283, -9.654002569019777 10.861587666490891, -6.613282429610447 10.223260780349227, -6.4029385141570545 7.182378292695237, -9.836853359751249 7.100441764640771))
+170	point_85	POINT (4.013660690397711 7.156232809180564)
+171	poly_85	POLYGON ((7.0595086949959915 4.004118280224513, 5.030078518809895 4.068378174782943, 4.491036181822643 4.3714666107934566, 4.807209414836568 6.679649336617814, 4.914731264734458 6.9328959989765595, 7.515214691475956 7.043998310009986, 7.0595086949959915 4.004118280224513))
+172	point_86	POINT (1.1593230554086655 6.4793129224965496)
+173	poly_86	POLYGON ((-9.078566156960921 3.9169907986080834, -9.178004346243794 4.219309370492121, -9.079157397343561 6.947224447245816, -8.832612529022295 7.275469798302406, -5.748606926067261 6.848656239334388, -5.843581462399192 4.285176230333099, -9.078566156960921 3.9169907986080834))
+174	point_87	POINT (-9.393467669306487 9.231933914792782)
+175	poly_87	POLYGON ((-5.26176993473056 -0.817602671394932, -8.266809040308829 -0.4073866098652239, -8.540964493758928 -0.3066534465793382, -8.801183108289745 -0.1718007854204187, -6.125309095163541 2.277000500599115, -5.384243459081748 -0.3189482503754124, -5.277024857780381 -0.7017344205335103, -5.26176993473056 -0.817602671394932))
+176	point_88	POINT (2.5905648586776553 -8.093185166096083)
+177	poly_88	POLYGON ((-0.2148338796455311 -1.5154924157402638, -0.9547182150397289 -0.6443335305513749, -0.294579677321775 2.3559901646698416, 3.0033284251164862 2.1263257861940037, 2.9992392995786603 -1.1222364746579947, -0.2148338796455311 -1.5154924157402638))
+178	point_89	POINT (-5.362008754340475 -1.0300641723994786)
+179	poly_89	POLYGON ((8.690014425933757 -3.279978837660893, 5.419658694526259 -3.0889533219312124, 4.973309116395216 -2.5024249067452704, 4.82783465881802 0.3299291209168444, 5.077991255640951 0.5664238839797993, 5.591751481444185 0.3545775881528481, 8.637830564577781 -2.5204986136540097, 8.690014425933757 -3.279978837660893))
+180	point_90	POINT (-6.566277351076955 -4.124032102106651)
+181	poly_90	POLYGON ((6.093701150963492 -5.2043119151880735, 3.1049600493415555 -5.016777550190401, 2.443238769360493 -4.642500672125651, 2.568367992866591 -2.2040394245888635, 5.574006589942166 -1.4818661432510774, 6.158072967795779 -2.201370659416182, 6.093701150963492 -5.2043119151880735))
+182	point_91	POINT (0.2992705801398223 1.006857607264954)
+183	poly_91	POLYGON ((-1.633539143420216 0.3732023554798554, -4.8607355531671805 0.414950215805705, -4.995468508095863 4.046409570655291, -1.4048626768308106 4.169546416708071, -1.2783563937949172 3.7432916852162488, -1.4378327313824806 0.845058774021807, -1.633539143420216 0.3732023554798554))
+184	point_92	POINT (-4.74085714088681 5.55287578863711)
+185	poly_92	POLYGON ((-2.5842840595289713 4.628170876527333, -3.1185089382858004 5.2606097633553475, -2.365525639222472 8.145275625710681, 0.0248899910731049 7.589301768089527, -0.1305229333811511 4.658555051204798, -2.5842840595289713 4.628170876527333))
+186	point_93	POINT (-1.0322684368957113 -0.208026948340517)
+187	poly_93	POLYGON ((5.80839156551537 4.607662329909914, 3.156187244304255 4.78687265291158, 2.6469118649355994 8.42368360937174, 3.581776281912167 8.371602319089144, 5.916079054547628 4.761755632264842, 5.80839156551537 4.607662329909914))
+188	point_94	POINT (-4.974113109827362 8.503981669171266)
+189	poly_94	POLYGON ((4.026068327381162 0.4941620285082604, 0.860703988115135 0.6980004144755996, 0.5859703792851723 0.8040200948028955, 1.20041195672033 3.490275578818, 4.268881472652459 4.227299092337475, 4.148557500755785 1.2055656152849736, 4.026068327381162 0.4941620285082604))
+190	point_95	POINT (-0.1569503214176793 0.3398464373429362)
+191	poly_95	POLYGON ((-6.834611991694976 1.770273317585172, -9.300076023809767 1.8005135998309791, -10.018837111723226 4.255162098609548, -9.847729107700387 4.845451404173282, -6.255909810575613 4.805909618960804, -6.078326460736232 1.8033043875149661, -6.834611991694976 1.770273317585172))
+192	point_96	POINT (-8.73671062469581 -10.26024914840585)
+193	poly_96	POLYGON ((-6.813940623370744 5.420917893668264, -6.910118948519615 5.600373828254127, -6.625341425992686 8.488139564650387, -6.551255881516405 9.123737977200332, -4.290074759201873 8.968033473585916, -4.087418329433863 8.380043708251442, -4.307376775294237 5.8395192044017366, -6.813940623370744 5.420917893668264))
+194	point_97	POINT (1.5282358289645352 -5.52490156676873)
+195	poly_97	POLYGON ((5.016863499945459 -9.464083482843623, 2.3098435127000867 -6.494104594394693, 2.5335742944008053 -6.125997005151787, 5.5365903844246205 -5.922555509052864, 5.397746556507554 -9.222657824865795, 5.016863499945459 -9.464083482843623))
+196	point_98	POINT (5.314923919724533 2.228742655203983)
+197	poly_98	POLYGON ((3.94623193505765 -3.28892067668792, 3.5728862406645066 -2.7416682426553787, 3.4118776024478588 0.0655538941020761, 4.105887409064918 0.3903251953526694, 7.170799136524596 -2.660189025982422, 3.94623193505765 -3.28892067668792))
+198	point_99	POINT (2.9797301592051104 2.881108141669112)
+199	poly_99	POLYGON ((-5.3860923295094505 -6.994357126602125, -5.627685292013884 -4.069388737181194, -5.506915003586234 -3.115426528442142, -1.8769654281109565 -3.752711097143165, -4.941427041314127 -6.679683575338314, -5.3860923295094505 -6.994357126602125))
+200	poly_m0	POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))
+201	poly_m1	POLYGON ((0 10, 1 10, 1 11, 0 11, 0 10))
+202	line_m2	LINESTRING (10 10, 11 11)
+203	point_m3	POINT (10 10)
+204	point_m4	POINT (-10 -10)
+205	poly_m5	POLYGON ((9 9, 11 9, 11 11, 9 11, 9 9))
+206	line_m6	LINESTRING (9 9, 11 11)
diff --git a/core/src/test/resources/spatial-predicates-test-data.tsv b/core/src/test/resources/spatial-predicates-test-data.tsv
new file mode 100644
index 00000000..bcd9e8a4
--- /dev/null
+++ b/core/src/test/resources/spatial-predicates-test-data.tsv
@@ -0,0 +1,907 @@
+0	poly_0	POLYGON ((-8.341045120860311 -5.62242508740254, -9.044126614177175 -5.618966706325255, -9.186303854001588 -4.677244370296111, -8.364762552575412 -4.746220796537222, -8.325644942940325 -4.760786314583519, -8.298440383327842 -5.572527985531895, -8.341045120860311 -5.62242508740254))
+1	line_0	LINESTRING (4.259416449213548 -2.8532709169732486, 4.364276221204584 -2.816782323784457)
+2	point_0	POINT (-3.804609561758885 -9.227800172299515)
+3	poly_1	POLYGON ((8.385878302237218 -6.809692089039363, 7.604800642628949 -6.750396817640533, 7.5482574276328025 -6.740977540831852, 7.4992511865253375 -6.678603222801474, 7.4520725264310075 -5.947146124999229, 8.28868956255955 -5.910956934009458, 8.385878302237218 -6.809692089039363))
+4	line_1	LINESTRING (0.7815693141383366 6.824949770613585, 0.8149999680305149 6.0381784097039874)
+5	point_1	POINT (-1.2824797365640914 4.588042562030753)
+6	poly_2	POLYGON ((2.687156858099715 7.166101036920338, 1.9035078751505146 8.075921495413915, 2.7229167575474715 8.140688616575751, 2.751153127239058 7.2561065921367405, 2.687156858099715 7.166101036920338))
+7	line_2	LINESTRING (-8.961173822708158 -3.93776773106963, -8.08908227533288 -3.8845245068873444)
+8	point_2	POINT (5.504449110391926 -2.1060970771922216)
+9	poly_3	POLYGON ((-5.059265173607508 2.4597577003740376, -5.064728419184651 3.356189506381222, -4.209580216578018 3.3333798823561778, -4.2255728446001575 2.566065249624219, -5.059265173607508 2.4597577003740376))
+10	line_3	LINESTRING (0.4745166795749912 1.9301776232814654, -0.3621163281419696 1.9664044040570228)
+11	point_3	POINT (-5.953120670505069 4.107737981860775)
+12	poly_4	POLYGON ((5.137011616348417 -8.563682322077854, 5.072600590136151 -8.543263646410818, 5.052798331625284 -7.642707469309771, 5.096005753793902 -7.674801379381424, 5.903873273131616 -8.48231839686209, 5.137011616348417 -8.563682322077854))
+13	line_4	LINESTRING (-8.75975960626321 6.030579041718783, -9.655916057497592 6.842450176982686)
+14	point_4	POINT (-6.3536387196724595 8.669411293374152)
+15	poly_5	POLYGON ((-3.4211180030028037 -9.538711419625779, -4.243504731404403 -9.53802795585015, -4.194574559013516 -8.704046320378568, -4.1060038011468585 -8.657750506152855, -3.351177828807136 -8.648432428025995, -3.322942869421644 -9.45089293978144, -3.4211180030028037 -9.538711419625779))
+16	line_5	LINESTRING (-4.035084408493117 1.7823139131745067, -3.2283948729717733 1.0244826338961297)
+17	point_5	POINT (-4.372796056963719 9.460662214914782)
+18	poly_6	POLYGON ((7.282727035261257 5.893206693935388, 7.214832360294231 5.918926909146094, 7.180658299841459 6.603725544929477, 7.233128513715139 6.66504551274461, 8.155752348112273 5.989432015827236, 8.084807600161469 5.956533873978985, 7.282727035261257 5.893206693935388))
+19	line_6	LINESTRING (-1.9839422575942238 1.9735741307092298, -2.0213729278921564 1.92840619940691)
+20	point_6	POINT (-4.862812398353427 -1.1975291647357136)
+21	poly_7	POLYGON ((0.9475564026667593 5.502523476406085, 0.0758848711710676 5.504474333376686, 0.1589356364888746 6.285533581060672, 0.9293298861959323 6.3687918529086005, 0.9552270603629477 6.241353693851264, 0.9475564026667593 5.502523476406085))
+22	line_7	LINESTRING (-0.52153913290265 -8.34706873782859, 0.1908380218821151 -8.389919104212607)
+23	point_7	POINT (2.9096390367227998 1.60232927698764)
+24	poly_8	POLYGON ((-4.278784453840437 -1.4708272571657508, -5.062237498534594 -0.5767840912968543, -5.088246639726636 -0.4840433360985055, -4.298398286919371 -0.5253735026265955, -4.278242077221902 -0.5921212065318148, -4.271222173128804 -0.634431149021122, -4.278784453840437 -1.4708272571657508))
+25	line_8	LINESTRING (-7.214132063019879 -8.798948934036371, -7.232218839784236 -8.87907280306588)
+26	point_8	POINT (-3.791764430474503 6.059974143767811)
+27	poly_9	POLYGON ((-1.1261118422622984 8.744959716303532, -1.9957653135626532 8.7871888586358, -1.9260846926032662 9.685043626315606, -1.0608600287359236 9.570190738063282, -1.0422323424060767 9.560755714332988, -1.044230128418475 8.8843357660407, -1.1098884567122922 8.76451854924099, -1.1261118422622984 8.744959716303532))
+28	line_9	LINESTRING (0.4302726470120163 9.641512964747282, 1.0980436822814412 9.63048078497016)
+29	point_9	POINT (-4.340411095398747 -5.505369399064048)
+30	poly_10	POLYGON ((0.2776992965094564 1.945702811228592, 1.0514256568674085 2.904239156281455, 1.1253970354592422 2.7968259473535113, 1.2078709819140752 2.1183227953008794, 1.126274066635715 1.9473636513798862, 0.2776992965094564 1.945702811228592))
+31	line_10	LINESTRING (-3.2332412294241792 -4.7187425541506025, -2.4741496367399396 -4.788358598694694)
+32	point_10	POINT (-6.100104486215135 8.116198532504695)
+33	poly_11	POLYGON ((7.301515135004183 1.5640572427221378, 7.17816150342235 1.6275483098488335, 7.184677121593924 2.278870566905244, 7.198885608642907 2.3429933339867604, 7.3446325265273815 2.353209897006164, 8.068919427769831 2.401029088350059, 8.014460336873851 2.2737525973794135, 7.301515135004183 1.5640572427221378))
+34	line_11	LINESTRING (-2.638564923564658 8.51861237517718, -2.586966555497543 7.593946103104851)
+35	point_11	POINT (5.479627960653181 6.905988092097981)
+36	poly_12	POLYGON ((0.1164543792571295 4.322480373130756, 0.1438051443594537 5.06918698985025, 0.1711124678587126 5.1530197733577205, 1.0429785953157409 5.138151652148315, 0.9484077155682228 4.987788925822409, 0.1164543792571295 4.322480373130756))
+37	line_12	LINESTRING (-5.742620801957906 -4.405110861618303, -6.557302674562932 -3.781832755528418)
+38	point_12	POINT (6.634409280273252 7.628152450151502)
+39	poly_13	POLYGON ((-1.341100066619353 -2.561753454656496, -2.272941024389553 -2.5145492312091413, -2.283264617153241 -2.488902774404881, -2.2902102160856614 -1.746691100712545, -1.4752255152348643 -2.4294019387933923, -1.341100066619353 -2.561753454656496))
+40	line_13	LINESTRING (5.924620939027833 -3.3863866280746304, 5.885039403964592 -4.253744110569072)
+41	point_13	POINT (-6.298363860226827 -5.364944712168557)
+42	poly_14	POLYGON ((6.350880143917798 4.330141826732806, 6.337021667480083 5.319163726711953, 7.186423266414368 5.293968429618893, 7.121856241468282 5.130922358807062, 6.399517126596475 4.341491249468908, 6.350880143917798 4.330141826732806))
+43	line_14	LINESTRING (-0.7169607971135469 1.9371956050186607, 0.2734406653090171 1.9287069292603936)
+44	point_14	POINT (7.307100645091645 10.08065625798863)
+45	poly_15	POLYGON ((2.9373469822329454 3.263990125305701, 2.888979415407886 3.302319002339507, 2.8109212409291473 3.4485441051057135, 2.967607592429269 4.153992457266921, 3.7281073383420735 3.4185135746359148, 2.9373469822329454 3.263990125305701))
+46	line_15	LINESTRING (-1.8344808037264353 -0.632577548136088, -1.8557956007688725 -0.4710989608959286)
+47	point_15	POINT (0.7133333186639641 -7.326163744021448)
+48	poly_16	POLYGON ((-0.9529038745181877 -7.195066192844958, -0.9134161304556478 -6.26391779386333, -0.7838381941795198 -6.265457296210764, -0.081133924576653 -6.369521857821131, -0.1043001684020768 -7.058607685294768, -0.9529038745181877 -7.195066192844958))
+49	line_16	LINESTRING (-7.042861336080368 3.607215741632449, -6.23437276417791 3.5819859622262746)
+50	point_16	POINT (-4.84102181383017 6.291167970899114)
+51	poly_17	POLYGON ((-6.518521938586374 3.9768258486651957, -7.437156826053486 4.088062112075494, -7.3445667634069665 4.757702494349495, -6.52436068992501 4.032284114574967, -6.513632564812705 3.9835758352578705, -6.518521938586374 3.9768258486651957))
+52	line_17	LINESTRING (-0.2566177381805626 2.5197619642032807, -1.0003591168098838 3.3974065946312484)
+53	point_17	POINT (-4.0292222552163866 7.415739071348301)
+54	poly_18	POLYGON ((0.7663555233653736 -10.013538930351713, -0.1878902034860146 -9.951365538142582, -0.0487835064327358 -9.1868042079945, 0.6506130555965626 -9.085334059904831, 0.763373219631726 -9.17631592771046, 0.7838206930169542 -9.972365807181722, 0.7663555233653736 -10.013538930351713))
+55	line_18	LINESTRING (-9.626898463086198 7.026756548321745, -8.726750357977192 6.9847131005497785)
+56	point_18	POINT (8.295657385692484 1.3607610384809188)
+57	poly_19	POLYGON ((8.802678163331352 -6.2105055456203635, 8.169068775068368 -6.156692067433866, 8.14363773416985 -6.1269427055861705, 8.127807408694377 -6.06898555767203, 7.996420781733409 -5.250964113265186, 8.896261390050567 -5.429021880593777, 8.840849381207459 -6.039309120548222, 8.802678163331352 -6.2105055456203635))
+58	line_19	LINESTRING (6.877074646305105 8.84304795125544, 6.8714290461006975 8.85158341890548)
+59	point_19	POINT (7.913218794102653 -7.027774474269595)
+60	poly_20	POLYGON ((-0.794635143511891 -2.2373238584087476, -0.6893515769958392 -1.3684303206678572, 0.0802816547654411 -1.3741548114081845, 0.1413716515244074 -2.2348102949435615, -0.794635143511891 -2.2373238584087476))
+61	line_20	LINESTRING (-6.427506160494745 -3.105029830162317, -5.613000920611421 -2.170787230635316)
+62	point_20	POINT (8.512091531423945 -1.8980169942079193)
+63	poly_21	POLYGON ((-8.34256388078175 -2.9779412008131825, -8.497851749117252 -2.239742214202642, -7.634284896937456 -2.1520206166715083, -7.606987677242696 -2.9420881940453762, -8.34256388078175 -2.9779412008131825))
+64	line_21	LINESTRING (3.816389205579431 -1.0787942453217885, 2.9138628851638324 -1.190717364349961)
+65	point_21	POINT (-9.636172683379437 -5.972553577640288)
+66	poly_22	POLYGON ((0.3569786377139333 -1.0617504999530603, -0.5142873101961043 -1.0049907016950226, -0.58278765149415 -0.9671908783339478, -0.6159352623698096 -0.2629612838806441, -0.5626168834219786 -0.1595171700342207, 0.2420859704909237 -0.23433586658474, 0.3569786377139333 -1.0617504999530603))
+67	line_22	LINESTRING (-0.6001701168376634 -1.1939453284278696, -1.4601282728781169 -0.4339267563810959)
+68	point_22	POINT (-2.143546187862654 0.9211767778233255)
+69	poly_23	POLYGON ((-0.8248508243947723 -4.742807547477747, -0.6967425733864737 -3.7761749579211172, -0.6607621517943152 -3.757188227687887, 0.0099394180817334 -3.814383643868721, 0.1357703938643683 -3.9438970277806167, -0.0141701692054749 -4.597537338917438, -0.8248508243947723 -4.742807547477747))
+70	line_23	LINESTRING (2.405073573843932 -1.4494546786669342, 2.4953863168047627 -0.7009301950220018)
+71	point_23	POINT (4.365595319401747 -6.157931595718839)
+72	poly_24	POLYGON ((-3.1461111494004226 -8.035254712331888, -3.7949644601750223 -7.884100452837585, -3.914551388443184 -7.179242061344779, -3.9017769955666894 -7.177111995257235, -3.0364648695819616 -7.209637154858898, -3.1461111494004226 -8.035254712331888))
+73	line_24	LINESTRING (-1.1989384224415625 2.082080022423184, -1.0628827028481145 1.358438231886408)
+74	point_24	POINT (-0.7514765422569063 6.450804181114841)
+75	poly_25	POLYGON ((-8.180328250249184 -7.045708226592582, -9.11979827953892 -7.03772000463645, -9.128943239820126 -6.316568869262979, -8.292488556629017 -6.225695054627733, -8.248217439930109 -6.242856990691303, -8.229119452215867 -6.335623787047183, -8.180328250249184 -7.045708226592582))
+76	line_25	LINESTRING (-8.660744267821302 5.177386319089599, -8.675778233234649 5.107030968109559)
+77	point_25	POINT (-6.140950584443284 4.851715705127745)
+78	poly_26	POLYGON ((10.009448275862928 4.4886831571642, 9.224613152912521 4.54568504240296, 9.329282867173708 5.326128626851834, 10.14242833394314 5.395554386355026, 10.195183203615139 4.6138874594009565, 10.173080260226625 4.5419345465368295, 10.009448275862928 4.4886831571642))
+79	line_26	LINESTRING (-7.308018864970235 -9.406851057486428, -7.344991699995828 -8.779741615048112)
+80	point_26	POINT (1.1697176061574925 9.146621378183594)
+81	poly_27	POLYGON ((-6.565698499212745 -4.626580463639897, -6.663040960524749 -3.8230077506582245, -6.525909605862492 -3.741127467921291, -5.83982381711216 -3.7539968759500253, -5.826545396675619 -3.805012495812069, -6.565698499212745 -4.626580463639897))
+82	line_27	LINESTRING (-3.9727770006304506 -4.583219522712756, -3.239682317432677 -4.693876791738792)
+83	point_27	POINT (-3.1155496136785454 8.894633649627544)
+84	poly_28	POLYGON ((8.044445000068553 -9.169729323384503, 7.315127516301282 -8.990509759048344, 7.188731919178507 -8.296796129392341, 7.970561342312025 -8.189693740766309, 8.045547901826728 -8.181353331827577, 8.156603147680126 -8.351059134975351, 8.044445000068553 -9.169729323384503))
+85	line_28	LINESTRING (-8.945695474383056 -3.337522731747069, -9.793894302471658 -4.017174240757391)
+86	point_28	POINT (3.6405281252604578 -7.139993103252537)
+87	poly_29	POLYGON ((-0.6077911399252718 5.241963768737231, -0.6518219684023263 5.387903996663612, -0.5625111261758342 6.033369648804421, -0.5427213124397957 6.0896029624392884, 0.1607662145651211 6.2012182918096235, 0.1314972580220636 5.348879665994348, -0.6077911399252718 5.241963768737231))
+88	line_29	LINESTRING (1.133078492770938 -3.131749144404472, 1.030739152405782 -3.141872943649054)
+89	point_29	POINT (-10.169424670857987 1.8755886235597934)
+90	poly_30	POLYGON ((-5.790263443218332 -1.029333490693044, -5.784683227854861 -0.2293009626162705, -5.617374448396491 -0.1417287892666628, -4.810085096534882 -0.9491123453247586, -5.790263443218332 -1.029333490693044))
+91	line_30	LINESTRING (-1.1233964843986144 -4.105918132469847, -1.1959701735694557 -4.984827948276702)
+92	point_30	POINT (-6.215150593657495 5.056193773423818)
+93	poly_31	POLYGON ((-6.8412435198653645 7.98458185329605, -7.565235986860834 8.059501245913118, -7.582356529394753 8.742030707501515, -6.753632605752661 8.812662762768, -6.687938210695092 8.079105130181498, -6.8412435198653645 7.98458185329605))
+94	line_31	LINESTRING (-5.731849061109386 0.6981247312152765, -6.459991288871876 0.8581623986401088)
+95	point_31	POINT (-2.4297951060595064 -3.5610770134763032)
+96	poly_32	POLYGON ((7.368373480809839 1.4368461182180319, 6.500546218091739 1.5038055836084205, 6.563925214098374 2.2589630628070343, 7.237280533013008 2.267449159848179, 7.298633662042227 2.2570479424913885, 7.3911799422133635 2.2206353088739936, 7.368373480809839 1.4368461182180319))
+97	line_32	LINESTRING (-9.016970273258899 1.0453873333611254, -9.010060719312122 1.694617228580392)
+98	point_32	POINT (1.3388201258973285 0.3442209516789338)
+99	poly_33	POLYGON ((-0.5555688840001587 8.855811176496447, -0.6529656767420278 8.879111161523987, -0.6655217698070728 9.704228230091015, 0.2036438298184179 9.722986134933986, 0.3095659313153699 9.683237571041717, 0.2780455245162982 8.931267129498613, 0.1888987782798426 8.912714073302325, -0.5555688840001587 8.855811176496447))
+100	line_33	LINESTRING (5.185489958053301 0.5292839282581288, 5.970467682398006 0.465941670769923)
+101	point_33	POINT (0.1506564948258851 4.211385310707256)
+102	poly_34	POLYGON ((-8.275596593307885 3.883518886865532, -8.959317850712056 4.051362291917166, -9.035684920105417 4.809063039069808, -9.018681374164462 4.845416845731721, -8.966118697031238 4.851748792205808, -8.250792524621527 4.789853199193143, -8.234802382292559 3.9847074479641535, -8.275596593307885 3.883518886865532))
+103	line_34	LINESTRING (5.739182132141206 2.9178622493532185, 6.643008160905857 2.131272489671543)
+104	point_34	POINT (-1.8967916353698582 -7.156716037064784)
+105	poly_35	POLYGON ((8.758034864625342 2.9980683915608695, 7.967249704390361 3.882534092081329, 8.054064449746269 3.9248735343401973, 8.798066518562674 3.972528636175738, 8.86081520419102 3.046072328757754, 8.758034864625342 2.9980683915608695))
+106	line_35	LINESTRING (-9.303743758544854 -7.941631579017222, -9.305039720990534 -7.304070249374269)
+107	point_35	POINT (6.670457095363657 -0.942146642828085)
+108	poly_36	POLYGON ((-0.2322827066308499 -10.286457498689796, -0.2512374408695764 -9.394454968846286, 0.558886732038588 -9.399035504987749, 0.6285901172323065 -9.434808760578468, 0.6977414898968327 -9.550033389997221, 0.7144237657651435 -10.233641688972464, -0.2322827066308499 -10.286457498689796))
+109	line_36	LINESTRING (-2.7113032875672305 -3.5289579858948037, -3.330923496767945 -3.4342686030515597)
+110	point_36	POINT (4.0509983463530785 8.715677304907945)
+111	poly_37	POLYGON ((4.225727160358487 -6.059553154215782, 4.049269309078438 -6.049983904404332, 3.2540582448259214 -5.992726340920092, 4.059579486157348 -5.2422793294505805, 4.230151298825547 -6.03711980669928, 4.225727160358487 -6.059553154215782))
+112	line_37	LINESTRING (-0.8339756807389545 8.081293584036382, -0.8423585404057932 8.858425499540777)
+113	point_37	POINT (-7.928313980735877 -7.203037923519054)
+114	poly_38	POLYGON ((-0.78576184232315 -2.1357768632076284, -1.5613709226370263 -1.363401171611216, -1.591037493880595 -1.3291729427190857, -1.6239612018665839 -1.211687471341362, -0.7822145970428193 -1.2441869147203575, -0.78576184232315 -2.1357768632076284))
+115	line_38	LINESTRING (5.063055404500314 -0.1317323806257239, 4.296081182795787 -0.9198546021420181)
+116	point_38	POINT (-9.506859023487454 3.5937830760139433)
+117	poly_39	POLYGON ((9.861683078430026 7.997799957652565, 9.089862292315484 8.13764662736992, 9.077957734552298 8.821442303783671, 9.896300497908605 8.977953704747765, 10.038726078248894 8.861419350424807, 9.937803557157219 8.098260517467493, 9.861683078430026 7.997799957652565))
+118	line_39	LINESTRING (-8.273909056581076 -8.992182043553258, -7.3074647800036825 -9.902334447012215)
+119	point_39	POINT (2.8875865765775695 -1.7689710963169625)
+120	poly_40	POLYGON ((7.224242109408361 2.761173929103348, 6.406255306202201 2.937983025880961, 6.506439180227179 3.673379911435223, 7.184821823674693 3.6744298145378926, 7.334278180907835 3.6597261206510776, 7.3097072685361875 2.8503855151231186, 7.224242109408361 2.761173929103348))
+121	line_40	LINESTRING (0.8005281578913626 -4.846217229522799, 0.1254211199976123 -4.815502029013477)
+122	point_40	POINT (10.10572625518535 -0.6029875171409507)
+123	poly_41	POLYGON ((-3.945736900072053 7.35391842989364, -3.979825300368167 7.419166032124831, -4.001866168381255 8.206981276062331, -3.158489011109208 8.104411661805875, -3.0640196962290407 7.3539412175213545, -3.945736900072053 7.35391842989364))
+124	line_41	LINESTRING (-9.442223155638024 2.348279588706857, -9.505628682456377 1.4138595636266125)
+125	point_41	POINT (0.7186184229662123 8.820681517773918)
+126	poly_42	POLYGON ((4.402535664396659 0.5038647065324449, 4.322985820436241 0.5911477514149599, 5.0565539884397035 1.3552864833514218, 5.147268097136518 1.41932876477871, 5.200477123399372 1.4071226929269878, 5.1334059949937405 0.5792849816194339, 5.084473564676028 0.5214298148983046, 4.402535664396659 0.5038647065324449))
+127	line_42	LINESTRING (-5.2051890760058885 5.18160426705268, -6.098206835462388 4.243544479833599)
+128	point_42	POINT (9.060638011664885 -1.8570380927812011)
+129	poly_43	POLYGON ((6.8256641186639735 -6.459268275031641, 6.687550702901976 -6.459191800612743, 5.986389008310732 -6.40612778661102, 5.937833485248722 -6.265464949247278, 5.994481751672932 -5.470541743195314, 6.701522138185892 -5.496206014938949, 6.793139617371336 -5.591471399767818, 6.8256641186639735 -6.459268275031641))
+130	line_43	LINESTRING (-0.9243909307702565 9.532012983870384, -0.8114385784650013 9.368142786645382)
+131	point_43	POINT (-2.9989337731717853 7.636719321345162)
+132	poly_44	POLYGON ((-5.3395574009634625 -0.2428886478024818, -5.983368698140604 -0.1852084898547302, -5.999877101437244 -0.1802544395600241, -6.13968598734435 0.5959219646519677, -5.991106433883647 0.6676102478806698, -5.315106263057165 0.5430995907609122, -5.300004966073571 0.5359656714813215, -5.3395574009634625 -0.2428886478024818))
+133	line_44	LINESTRING (-2.1811766496973193 -0.0809460692571112, -2.2516265186160256 -0.9675924895661714)
+134	point_44	POINT (-4.005592731854894 3.0048503121820747)
+135	poly_45	POLYGON ((-7.197886549198803 -4.869357297807559, -8.017054828047547 -4.820615099266362, -8.102459104568993 -4.78617925653798, -8.08901411137091 -4.096656677346615, -7.270222494706546 -3.982005793111245, -7.163469863400418 -4.8197960724316875, -7.197886549198803 -4.869357297807559))
+136	line_45	LINESTRING (-6.678382251066877 -6.250107931683308, -6.75290965840691 -5.570597197234245)
+137	point_45	POINT (1.876600842429857 -3.2314690027471333)
+138	poly_46	POLYGON ((-2.415187100742235 -8.42487846816219, -2.4446290503693415 -8.399603587730807, -2.5970312036362424 -7.512050963317062, -2.5232011741913385 -7.51679045442278, -1.7425340679815444 -8.31686925220078, -2.415187100742235 -8.42487846816219))
+139	line_46	LINESTRING (-9.62106675758407 -1.6211241049547787, -9.6308097663484 -1.6941966031658569)
+140	point_46	POINT (-7.525604212092424 -4.886410580665004)
+141	poly_47	POLYGON ((8.98292894591395 -3.50038745536647, 8.184114783716424 -2.6578060309069937, 8.165154601524158 -2.631883601714543, 9.032971386824709 -2.570619344077479, 8.996135856350042 -3.3275881234516995, 8.98292894591395 -3.50038745536647))
+142	line_47	LINESTRING (-6.455831952048082 -0.4989941745856303, -7.284166856214296 -0.36448903664969)
+143	point_47	POINT (6.60047661495815 -6.113729311819062)
+144	poly_48	POLYGON ((-0.5270246976275701 -3.540322667843437, -0.5657792264749771 -3.5063059585137353, -0.5709258208829906 -2.568480600595919, 0.3641580847903402 -2.709081203770831, 0.360685177507564 -3.3959654444393794, 0.2581351429997148 -3.440145052792524, -0.5270246976275701 -3.540322667843437))
+145	line_48	LINESTRING (-0.718496073926665 5.576722858284432, -0.7024647882318351 6.315754949172837)
+146	point_48	POINT (-4.297732415192593 -0.3493621863258255)
+147	poly_49	POLYGON ((-1.6975495543996337 2.34841186955929, -1.7884500289413265 2.423962226149278, -1.857674020675494 3.2778553238954498, -0.9573336520683422 3.146563488933843, -1.0276635399778113 2.3499473277732816, -1.6975495543996337 2.34841186955929))
+148	line_49	LINESTRING (-6.797054367617259 -0.4292269297933753, -6.7937824892856415 -1.2128710440796298)
+149	point_49	POINT (0.6272015431143259 -1.7349666210728742)
+150	poly_50	POLYGON ((4.680546216263917 -1.009425126072295, 4.659952902549861 -0.8452391303073765, 4.663982315751185 -0.1376652733530189, 4.77303383819982 -0.0851793092584571, 5.424153005882568 -0.1280193450999926, 5.574321781041512 -0.8931242032030042, 5.577594564508542 -0.9724700520245517, 4.680546216263917 -1.009425126072295))
+151	line_50	LINESTRING (4.0312049092275855 1.8917512152456637, 3.898833914448679 1.886068515645672)
+152	point_50	POINT (0.3690530191899353 -4.114730754613613)
+153	poly_51	POLYGON ((4.249548696881448 7.614678669991228, 4.173999578418609 7.632555219480244, 4.204136664633943 8.551288584381572, 4.272291372702076 8.548743701878477, 5.019011153806625 8.431057935013323, 5.094039640080483 8.391663180486171, 5.102082206481438 7.63975541139756, 4.249548696881448 7.614678669991228))
+154	line_51	LINESTRING (-1.6631966751885732 4.536018737286001, -0.8612892773834413 4.571207983760702)
+155	point_51	POINT (-4.819963171227652 -6.9600270003341524)
+156	poly_52	POLYGON ((-7.2300130347147755 -2.0197958901227606, -8.059936314179575 -2.0131451274660916, -8.087538160669324 -1.210772503752896, -7.961696547796858 -1.127860712872431, -7.903105512301064 -1.1594754318632592, -7.2300130347147755 -2.0197958901227606))
+157	line_52	LINESTRING (4.211985167892334 8.837678515961299, 4.251525584308152 9.666487293230912)
+158	point_52	POINT (2.0910574734594407 -9.97408873507408)
+159	poly_53	POLYGON ((-0.3520902561297732 -2.390442400485621, -0.3589451451295988 -1.4655019074915412, -0.3407516956137142 -1.4430662116927562, 0.4047847399273047 -2.2626699074064556, 0.4171606235858649 -2.3557673461138853, -0.3520902561297732 -2.390442400485621))
+160	line_53	LINESTRING (1.603323550472156 2.745351013123953, 0.8552962739037853 2.6332906309357487)
+161	point_53	POINT (-7.23541858943094 -6.732185795118087)
+162	poly_54	POLYGON ((1.1326841356411326 3.6126051283438207, 0.2146844685764113 3.7582530558027325, 0.2386637645913334 4.531286981439142, 0.3035709179185722 4.551925610560163, 0.9792712754571834 4.580220921628027, 1.1326841356411326 3.6126051283438207))
+163	line_54	LINESTRING (-8.061993435565126 -4.973207956304, -8.968960206640634 -4.850700118634232)
+164	point_54	POINT (-8.706847521312561 -8.058266650044619)
+165	poly_55	POLYGON ((-8.216512560568564 -3.667005854519789, -8.292240024472612 -3.5699412716620214, -8.39403001188132 -2.729460611884112, -7.460601584831859 -2.694213728940882, -7.49873043061649 -3.578225275780002, -8.216512560568564 -3.667005854519789))
+166	line_55	LINESTRING (-9.492029593245071 3.466642749697021, -9.439047685355629 4.3893333059698305)
+167	point_55	POINT (-7.947546428037024 9.39339132122927)
+168	poly_56	POLYGON ((-5.712653798367115 -8.430915368469117, -5.797841953641421 -8.299271612152069, -5.709334424569365 -7.530776314120694, -4.869747195303462 -8.329805110961889, -4.948241125162273 -8.345714394924666, -5.712653798367115 -8.430915368469117))
+169	line_56	LINESTRING (-5.186061171125647 -4.606935169928609, -5.082308948031734 -4.658230295295867)
+170	point_56	POINT (0.3292621369969908 1.1556125037728073)
+171	poly_57	POLYGON ((2.7389128503471682 7.060416363164504, 1.9993570144534976 7.67381322533416, 1.9251614541078172 7.818792467856509, 2.795804894170349 7.869968282617322, 2.8918742117984704 7.846306840563106, 2.8895732843286055 7.6758747200714055, 2.7389128503471682 7.060416363164504))
+172	line_57	LINESTRING (-9.187742997875427 2.7264987220814834, -9.239365423808776 3.694993877967923)
+173	point_57	POINT (4.757785229544427 -4.284804167989272)
+174	poly_58	POLYGON ((-10.240122168300458 2.0889534799798186, -10.29583477112089 2.1412978040737873, -10.22549087797641 2.263102710921429, -9.421492309522394 3.0210373288502135, -9.340146445952278 3.01169834686647, -9.337046109367646 2.227711008999329, -9.493345710272227 2.1122999879799167, -10.240122168300458 2.0889534799798186))
+175	line_58	LINESTRING (-0.3008706786049468 4.567015650616763, -0.1808163937652967 5.243440770112725)
+176	point_58	POINT (5.205883281384808 3.9471966836873844)
+177	poly_59	POLYGON ((9.229199489393297 6.22749422426813, 8.424710001126016 6.250071611123948, 8.323342140729212 7.056942830960991, 8.452475630799203 7.129572640185247, 9.262813823147875 7.127262409911749, 9.286891901266014 6.30908906973463, 9.229199489393297 6.22749422426813))
+178	line_59	LINESTRING (6.778840247991607 0.0375298260000466, 6.084094714372208 -0.0961581822862457)
+179	point_59	POINT (5.706855805080051 5.235856202798734)
+180	poly_60	POLYGON ((-8.669931017643178 -3.867601866362942, -8.679895879990816 -3.7855246316021165, -8.615523779281151 -3.137228719368466, -8.592031897373944 -3.0528473691854154, -7.73272377222095 -2.9677379483190487, -8.500414114904908 -3.830563267852486, -8.669931017643178 -3.867601866362942))
+181	line_60	LINESTRING (7.302557594887644 1.6467379212459972, 7.332385896664576 1.5578371924468555)
+182	point_60	POINT (-1.82938361490753 9.488305347629929)
+183	poly_61	POLYGON ((-2.453345417916406 -2.8223897077314177, -3.209574973518986 -2.807454234628266, -3.3280344933387274 -1.9434242909593764, -2.5642562390857258 -1.8973422311055053, -2.4408373326633352 -2.805904438599808, -2.453345417916406 -2.8223897077314177))
+184	line_61	LINESTRING (-4.059378189836082 4.625058077424756, -3.2371533450862433 4.619100652528182)
+185	point_61	POINT (-8.339316404134989 -5.451985726090552)
+186	poly_62	POLYGON ((-4.440760102788228 -4.236828397526104, -5.235385384577232 -4.117367747934101, -5.383355524833937 -4.073978952724184, -5.3155580348996265 -3.3333285738837226, -5.2266506048024635 -3.282450580353302, -4.440261144065487 -3.3727974961890297, -4.42828004964073 -4.1254007651024525, -4.440760102788228 -4.236828397526104))
+187	line_62	LINESTRING (-1.4067893441135966 -6.749128210515613, -1.2593525900374583 -6.77976035584045)
+188	point_62	POINT (-7.265892771623562 -2.007447664256961)
+189	poly_63	POLYGON ((-7.8909996294490234 2.9630700065280497, -8.640556008929291 2.9883148537004383, -8.650913451178516 2.9942195009332453, -8.704697413585647 3.803441486343154, -7.758731431103342 3.762308791670392, -7.744837090483299 3.080061075631145, -7.8909996294490234 2.9630700065280497))
+190	line_63	LINESTRING (-2.7581608316466517 -0.2697047236158089, -3.5727633312292317 -0.1586055521372147)
+191	point_63	POINT (1.4402639459144753 -7.4933682767011796)
+192	poly_64	POLYGON ((-6.806135296642088 7.681736040801685, -7.502558672005453 8.478451929896401, -7.564617719656278 8.625151173409844, -6.74709188681958 8.65265144047517, -6.741021817773019 8.62077210494352, -6.725029433493013 7.7580921538639895, -6.806135296642088 7.681736040801685))
+193	line_64	LINESTRING (3.81261322898722 -3.225083336879591, 4.678536094697047 -3.3149218835153733)
+194	point_64	POINT (9.070499245711881 -5.502454468812701)
+195	poly_65	POLYGON ((-0.9927264422038564 -10.350416587166643, -1.008902041681475 -10.19408187065969, -0.8952604274234125 -9.47391021341764, -0.2010748818696306 -9.397055908626871, -0.0846630199937324 -9.445946691174553, -0.0835258073796964 -9.562004558553935, -0.1593721446420147 -10.193297047003947, -0.1829382868460086 -10.309793571656245, -0.9927264422038564 -10.350416587166643))
+196	line_65	LINESTRING (-0.76495639482798 -9.319326877498881, -1.5586410704114553 -10.266961361548079)
+197	point_65	POINT (5.326619281815864 -1.4690484795554188)
+198	poly_66	POLYGON ((-8.567995574026272 5.434632196633414, -9.459607014590182 5.508822144214484, -9.46485875305969 6.198319360227298, -8.65373677971964 6.3175607817993065, -8.51577867919586 6.193875864008849, -8.559796464560371 5.497804720764585, -8.567995574026272 5.434632196633414))
+199	line_66	LINESTRING (0.5978205330820234 0.1435894323379348, 0.5474808696143527 0.8583564659828451)
+200	point_66	POINT (6.920329839540207 7.346724681437992)
+201	poly_67	POLYGON ((10.087275091417457 2.0344896346801637, 9.348246775500947 2.0664963300925043, 9.217251126986342 2.195453956464714, 9.363136981099927 3.003333314419397, 10.049922547167908 2.953494052470057, 10.087275091417457 2.0344896346801637))
+202	line_67	LINESTRING (-1.9100412594888942 5.6001338442193545, -1.1116317101896378 4.911111891681722)
+203	point_67	POINT (10.335333009278022 5.945257649662045)
+204	poly_68	POLYGON ((-8.149307111244479 -4.843271344652191, -8.792688599611573 -4.76324665867228, -8.967332186256613 -3.9597982666400107, -8.049928282206464 -3.973862428357964, -8.064660137412366 -4.669671501692293, -8.149307111244479 -4.843271344652191))
+205	line_68	LINESTRING (-6.181729487943899 1.4848663816771488, -6.124233273043159 0.6033473656861041)
+206	point_68	POINT (-6.623152116738473 -3.2573629474881987)
+207	poly_69	POLYGON ((-9.311998528966514 -6.4335612237446, -10.089448941792174 -6.3118356385170955, -10.040069310212784 -5.499331152262196, -9.361074404850367 -5.491565039415596, -9.306894404782469 -5.504814736587128, -9.23384389359395 -5.595813297651189, -9.185113961832137 -6.414298993778861, -9.311998528966514 -6.4335612237446))
+208	line_69	LINESTRING (2.962357945518473 4.733324004354694, 3.049238640862677 4.638384915547773)
+209	point_69	POINT (5.5851372450455266 3.7307283646142486)
+210	poly_70	POLYGON ((-7.85625570752883 4.835790093122092, -8.002593780257474 4.910581195199164, -7.985077979319908 5.594168913793657, -7.22007599024137 5.69795786274519, -7.149398404957536 5.6566200416551125, -7.08751195730631 5.558784005140981, -7.118857278649579 4.875571567314903, -7.85625570752883 4.835790093122092))
+211	line_70	LINESTRING (-9.501031340630155 8.955264902581979, -9.521925699684969 9.859150054377107)
+212	point_70	POINT (-7.19424660768211 -3.995108303459048)
+213	poly_71	POLYGON ((-2.894995317519117 6.526895176413907, -2.8717251663479404 7.486521288160526, -1.9975220432107172 7.3544616746469975, -1.9886966713377543 6.55842727098935, -2.894995317519117 6.526895176413907))
+214	line_71	LINESTRING (-7.283139378982936 -2.9916239143001446, -6.521995036278808 -3.1567915509150266)
+215	point_71	POINT (-0.5351945340773429 -7.424843113186315)
+216	poly_72	POLYGON ((-9.46149486214988 -3.4381409269736998, -9.4865832252604 -3.426873930518103, -9.500029174802288 -3.4125127292328727, -9.413301157569396 -2.556375988680394, -8.61596968432319 -2.646323711113473, -8.607703933547452 -3.409439665720397, -8.744710189865428 -3.437100450379144, -9.46149486214988 -3.4381409269736998))
+217	line_72	LINESTRING (-3.8850070392651403 -2.8791742950385504, -3.971539342666754 -2.726737043830533)
+218	point_72	POINT (-9.234797192597997 2.89024195852059)
+219	poly_73	POLYGON ((0.6883953807606851 -5.236257838308003, 1.5028369357475237 -4.357832840001357, 1.6310397262415612 -5.23604398059691, 0.6883953807606851 -5.236257838308003))
+220	line_73	LINESTRING (1.2748315339214782 -7.728086883879794, 1.2532037662434885 -7.561197566011092)
+221	point_73	POINT (1.2709706877721088 8.560869990541338)
+222	poly_74	POLYGON ((-0.7292636806351244 2.5283532408087055, -0.7891257997643241 3.34915500044798, -0.75930068958766 3.3930653490929172, 0.0146597069700067 3.353584975447659, 0.1144456348360052 3.3426462123357465, 0.0292418311723486 2.549569528157047, -0.7292636806351244 2.5283532408087055))
+223	line_74	LINESTRING (5.658771785060952 3.098556389695127, 4.81690647679919 2.261177470554268)
+224	point_74	POINT (-1.563663621492796 3.2455252681463307)
+225	poly_75	POLYGON ((4.457291273830926 -6.802813244006065, 4.3588800356823185 -6.051212218645143, 4.346692322069306 -5.9447026071520614, 5.172470937015192 -5.968143480361699, 5.170474837159271 -6.013725068133745, 4.502416515765547 -6.7790377798308885, 4.457291273830926 -6.802813244006065))
+226	line_75	LINESTRING (6.184615911815201 6.577168038054948, 6.0169400328157145 5.608030184184182)
+227	point_75	POINT (3.499099745708544 -5.374202790879755)
+228	poly_76	POLYGON ((-3.3872736879484924 -0.8332807170329799, -4.124759077163152 -0.6597957083545993, -4.2215001684553055 0.1371455461421532, -3.410057936518846 0.1233080640624364, -3.388312284454659 0.0405321724674008, -3.3872736879484924 -0.8332807170329799))
+229	line_76	LINESTRING (7.419069511482402 7.9971928778982715, 7.514481170076655 7.2368105165907615)
+230	point_76	POINT (5.441003845496888 7.492069572928307)
+231	poly_77	POLYGON ((5.344881391384688 -8.902391664747393, 5.273617791449155 -8.743117748482442, 5.287439980646372 -7.930170733923665, 5.399753460289903 -8.045784587819469, 6.100470557693867 -8.771695553203633, 5.344881391384688 -8.902391664747393))
+232	line_77	LINESTRING (7.559919476138089 -0.7179229152416909, 6.722327811659583 -0.6691914579825504)
+233	point_77	POINT (7.785859328614016 5.258276465279753)
+234	poly_78	POLYGON ((-0.7692679155711173 7.370316796768389, -1.5346735564337384 7.418986467897471, -1.5841448699294154 7.439500448711015, -1.6055132453617915 8.266044648333837, -0.702208620504791 8.188326963817765, -0.6364299298481111 7.466658363842332, -0.7692679155711173 7.370316796768389))
+235	line_78	LINESTRING (-3.9811461346992187 0.833616554416752, -4.036714513687146 1.511101647123029)
+236	point_78	POINT (6.607564271618715 -7.366810356368848)
+237	poly_79	POLYGON ((0.9879720384823407 6.391396688408228, 0.9419286796854346 6.475173674022437, 1.0438405874241352 7.265000007352936, 1.7644151347500912 7.321340027351045, 1.8378931938924263 7.1440877535609415, 1.7437772685027983 6.393277777205756, 0.9879720384823407 6.391396688408228))
+238	line_79	LINESTRING (-3.4233476599640893 4.45799756516801, -3.430623316598334 5.41146344653103)
+239	point_79	POINT (2.6643523327453127 -9.540256859493164)
+240	poly_80	POLYGON ((6.310922927365548 -2.086979255412403, 6.266552442046966 -2.07146319225396, 6.22013020285159 -1.0974260676155156, 7.060694483660607 -1.1013124542645811, 6.310922927365548 -2.086979255412403))
+241	line_80	LINESTRING (-8.746978361858673 3.5673845364666326, -7.863802675543437 2.8085057030952765)
+242	point_80	POINT (-0.2477959492931517 -1.6437520971554003)
+243	poly_81	POLYGON ((-4.476242069615431 -4.870349751481676, -4.505918781978444 -3.9942971816741215, -4.40499494248608 -3.944656739461109, -3.613093556111179 -3.9291385073246605, -3.6055376576513973 -4.789928590609073, -3.7796751819131833 -4.829636354050912, -4.476242069615431 -4.870349751481676))
+244	line_81	LINESTRING (-9.523134276386514 0.3711262591532178, -8.882214718720014 0.343718851671804)
+245	point_81	POINT (-6.545975671183689 3.701506038066384)
+246	poly_82	POLYGON ((9.18747616937847 6.8408861914450885, 9.083208560825303 6.880209261326662, 9.085178435151562 6.966212924241669, 9.135920210165159 7.721247783942551, 9.892686038483086 7.716775233937939, 10.053344392886046 7.7035612120206345, 10.039875524203389 6.860974667371115, 9.18747616937847 6.8408861914450885))
+247	line_82	LINESTRING (-8.295891191025605 4.454832757010461, -8.97706885114036 4.457854647802611)
+248	point_82	POINT (-6.7679535231024746 -8.029999705974017)
+249	poly_83	POLYGON ((-2.8066775046050303 -9.834473612572666, -2.039249537432176 -8.853621876494532, -1.9324397169321128 -8.887867308335045, -1.9273471407148355 -8.910842157829196, -2.0172561918674283 -9.661389565856624, -2.707600598828698 -9.830413844990025, -2.8066775046050303 -9.834473612572666))
+250	line_83	LINESTRING (-1.0719793058980982 -2.338498716322051, -1.1856397057317665 -2.2489592808075254)
+251	point_83	POINT (5.406812863458252 4.630679296521913)
+252	poly_84	POLYGON ((-5.915189922214178 -1.729578852447224, -6.723892904455576 -1.6561994922055632, -6.7657608818287915 -0.8597921510610218, -5.985556713040164 -0.8741062112111668, -5.8765987330940685 -0.9303899270646958, -5.915189922214178 -1.729578852447224))
+253	line_84	LINESTRING (-10.203637701407496 7.226306680923491, -10.113243160178296 8.073919667241556)
+254	point_84	POINT (1.224392461527318 4.911239905142045)
+255	poly_85	POLYGON ((-3.7869284567681336 1.9732906856750028, -3.844570930018328 2.751889379835454, -3.7823111001827936 2.852479783984465, -3.0904855588496494 2.8906311447822373, -2.8988797794130754 2.82675476839075, -2.934113238682269 2.101457310786175, -3.7869284567681336 1.9732906856750028))
+256	line_85	LINESTRING (3.119352671297817 6.607572112169374, 3.980005962443188 6.7354316422552785)
+257	point_85	POINT (-10.347047098933055 1.5175547747180504)
+258	poly_86	POLYGON ((8.353702340158447 4.373457119671067, 8.3228671251007 5.251619577106384, 9.148038611828811 5.310077094072023, 9.307034771231372 5.163985750247439, 9.127553367263925 4.439550400817443, 8.353702340158447 4.373457119671067))
+259	line_86	LINESTRING (-6.1959055222341 8.564360961302521, -5.350271156944791 9.334375911399357)
+260	point_86	POINT (-7.331335096047563 -1.1813730340945163)
+261	poly_87	POLYGON ((-8.47815148581127 -3.3663791848957274, -9.296623134626683 -3.2086033237823925, -9.245770836899071 -2.541241850905965, -8.410523340776164 -2.4272723638708245, -8.339045109820376 -2.4982798241904187, -8.404563651608711 -3.353365070359868, -8.47815148581127 -3.3663791848957274))
+262	line_87	LINESTRING (6.507623948261226 3.3069058854066875, 6.568878382609909 2.426868745419647)
+263	point_87	POINT (-3.128311409730141 -3.1887493239745153)
+264	poly_88	POLYGON ((8.40347658825089 -4.326764579759027, 8.27655966763955 -3.5532676510887526, 9.153799267196515 -3.535834221872596, 9.173342039010157 -3.557941751337342, 9.19241011278845 -3.597133197127276, 8.40347658825089 -4.326764579759027))
+265	line_88	LINESTRING (-9.985780151757346 5.476164174757885, -9.858499171329512 4.632976798581479)
+266	point_88	POINT (-8.343055808230872 0.1316559049261067)
+267	poly_89	POLYGON ((-0.1298909904541186 -10.081211998148834, -0.9355483246191909 -10.064536543833846, -0.9416912273802793 -9.114311292949866, -0.0221778080921428 -9.128664411330936, -0.1298909904541186 -10.081211998148834))
+268	line_89	LINESTRING (6.420932144846893 -7.002875189119181, 5.5971079155762995 -6.199965078599182)
+269	point_89	POINT (9.915091674511334 -1.379565415120871)
+270	poly_90	POLYGON ((0.1177171760941357 7.15665546441776, 0.1391180831672787 7.914464115204492, 0.2181187048697954 8.05087571009187, 1.0561540017353495 7.97959022045531, 1.054420348756195 7.215172021997013, 0.1177171760941357 7.15665546441776))
+271	line_90	LINESTRING (-8.38630437992374 -6.017513811685421, -8.39334111244637 -6.712113881476262)
+272	point_90	POINT (-8.511206462402544 9.81362280117268)
+273	poly_91	POLYGON ((5.472444756930448 0.3660487412328743, 5.391736970944341 0.399843152911205, 5.360504932141821 0.4661438203375027, 5.384052442942454 1.1940709824226885, 6.201039978585317 1.2149150404691897, 6.268026236444254 0.4754905286621331, 6.150935027027972 0.4558159741212032, 5.472444756930448 0.3660487412328743))
+274	line_91	LINESTRING (-8.839016006508219 -0.3178883547104156, -8.730133671155683 0.4193091571524842)
+275	point_91	POINT (6.535326251511064 -8.919806891544031)
+276	poly_92	POLYGON ((1.782277464242144 -7.348706346390101, 1.804662159921004 -6.546197317216791, 1.8477710304616255 -6.503312575130412, 2.6371829826445086 -6.478138610262983, 2.737179953614466 -7.223679823959049, 2.7134647652137973 -7.315070841191605, 1.9329035390196239 -7.344434543161943, 1.782277464242144 -7.348706346390101))
+277	line_92	LINESTRING (3.2101709856945857 3.2175618340802377, 3.3212906441473815 4.168141875393604)
+278	point_92	POINT (6.114198538480053 1.3440923904864324)
+279	poly_93	POLYGON ((-0.0054667615839828 -6.507429995803374, 0.1216458608503744 -5.731038892940065, 0.7851087379609132 -5.616289121088047, 0.9470905556070517 -5.656832159541457, 0.964929500926707 -6.45801693628635, -0.0054667615839828 -6.507429995803374))
+280	line_93	LINESTRING (-2.110202111836027 2.9157621536998315, -2.1385667091274354 2.8285147518076363)
+281	point_93	POINT (-0.8269809788662652 8.401691960165518)
+282	poly_94	POLYGON ((9.337813092363636 4.482108825519779, 8.394817142552997 4.50401411693719, 8.407110639490622 5.447825979280094, 9.246154986545157 5.365271421428994, 9.348322328165288 4.573040114971569, 9.337813092363636 4.482108825519779))
+283	line_94	LINESTRING (4.477875941099302 8.041910669657517, 4.483176589839662 8.734029641479983)
+284	point_94	POINT (8.90103299923698 2.771710145219749)
+285	poly_95	POLYGON ((7.169169064115412 -0.1997615986434794, 6.372045291066974 -0.1638819847051533, 6.395003259954446 0.6590663888803494, 7.281599660910063 0.5842394947212419, 7.301627309850401 0.5315484076075724, 7.169169064115412 -0.1997615986434794))
+286	line_95	LINESTRING (-5.832942699270196 -4.064438892477202, -5.73726714888771 -4.906817117662066)
+287	point_95	POINT (1.8563928712073823 6.412030622302766)
+288	poly_96	POLYGON ((5.312474711100349 3.7151129748067118, 5.269929046379006 4.561855525834848, 5.273854626380064 4.620609776163228, 5.29977081776548 4.662244814259153, 6.239147760103196 4.584760927026966, 6.257256923423932 4.538909622701113, 6.0937669033740915 3.7903346257723394, 5.312474711100349 3.7151129748067118))
+289	line_96	LINESTRING (-4.778306946283665 7.21590377050717, -3.9228357193275154 7.236270071885098)
+290	point_96	POINT (3.8269159703157984 6.932180531818723)
+291	poly_97	POLYGON ((10.175700694730606 -0.9466554143513026, 9.425517191201433 -0.9326882383276833, 9.375838854018347 -0.9090720453287794, 9.259891229664818 -0.0919212993746587, 9.36643136030966 0.0333027802297195, 10.193113323478354 0.0310940301004236, 10.238472757968092 -0.1561048092697717, 10.175700694730606 -0.9466554143513026))
+292	line_97	LINESTRING (-2.7165428571140318 -7.836343567221222, -1.905233824280366 -7.841550307690365)
+293	point_97	POINT (-5.040872307111143 -1.3688984104398205)
+294	poly_98	POLYGON ((0.5026302933884075 8.132091262361921, 0.4540657003640989 8.312253908963674, 0.5972323672416346 9.08503833319398, 0.6501329995511904 9.031857886114274, 1.3339536580217248 8.278110269444307, 1.3659005754369402 8.211904370376601, 0.5026302933884075 8.132091262361921))
+295	line_98	LINESTRING (-2.9529189380165275 6.1980419952989285, -2.9478295799487286 7.070136144301936)
+296	point_98	POINT (1.7821517237415616 6.325432454313146)
+297	poly_99	POLYGON ((5.158911933735025 -1.5633652488500487, 5.189970074861842 -0.6336805614800461, 5.925847946172807 -0.7673458075647984, 5.158911933735025 -1.5633652488500487))
+298	line_99	LINESTRING (-8.621944147586554 7.774470669247517, -9.389722460573584 8.589563912282202)
+299	point_99	POINT (-0.0757537402184204 -9.298291123731646)
+300	poly_100	POLYGON ((5.7844997407995296 -5.992019317174171, 5.038523188877106 -5.87832964248807, 5.078423192250668 -5.10805437338795, 5.89689471950166 -5.090678844736859, 5.856398127457679 -5.978318831658482, 5.7844997407995296 -5.992019317174171))
+301	line_100	LINESTRING (8.997666197857264 -9.053530903952023, 9.080420020792953 -9.857106041029075)
+302	point_100	POINT (-1.9250203356328552 5.653557227082372)
+303	poly_101	POLYGON ((-10.019927412903598 2.2132426678812998, -10.135494460680077 3.112879374955168, -9.332866462226804 3.043660235001145, -9.167726631329026 2.257642798709749, -10.019927412903598 2.2132426678812998))
+304	line_101	LINESTRING (2.411013387215278 -8.71318564763373, 1.5403108020295344 -8.646884911596093)
+305	point_101	POINT (1.7433415989173018 6.506058889569134)
+306	poly_102	POLYGON ((4.91352713696018 7.292330093223186, 4.978446643920013 8.187577034966637, 5.833934650392632 7.428742846042481, 5.834931207673213 7.368844151008607, 5.7518398702087215 7.323770265926224, 4.91352713696018 7.292330093223186))
+307	line_102	LINESTRING (-6.782197203397314 -3.5010405056763467, -6.776169189803573 -2.643046621411339)
+308	point_102	POINT (8.4041776112082 9.099628206754067)
+309	poly_103	POLYGON ((6.982960662634852 -6.933772614625324, 7.808165363379873 -6.001237882923155, 7.905296363339381 -5.957448835050144, 7.911113377670185 -6.794148673104345, 6.982960662634852 -6.933772614625324))
+310	line_103	LINESTRING (-4.882481430329063 3.098095933325064, -5.733634007570111 2.351182383881741)
+311	point_103	POINT (-7.407918394485915 5.17736072142787)
+312	poly_104	POLYGON ((2.3585204371692723 -7.628644754908686, 1.5352707488063557 -7.590893832390041, 1.5483169060153439 -6.702120569719848, 1.553362515494187 -6.688270674778196, 2.324921584371593 -6.63403815450957, 2.422362084025589 -7.523096968416073, 2.3585204371692723 -7.628644754908686))
+313	line_104	LINESTRING (4.973837670099232 -2.137022636510225, 5.006694744389781 -1.4577627363451584)
+314	point_104	POINT (5.22128163680383 6.826018563028705)
+315	poly_105	POLYGON ((-8.284423999258795 2.2881616687268087, -9.125635910792816 2.3093897026322354, -9.133837921910734 3.0404931631849887, -8.177827626490306 3.1606791253209003, -8.284423999258795 2.2881616687268087))
+316	line_105	LINESTRING (-0.3922383848375751 -9.375033235829543, 0.4459588224471472 -9.35529602753229)
+317	point_105	POINT (7.571857128196066 2.329452520998995)
+318	poly_106	POLYGON ((-5.8749988067684304 -2.357630962197754, -6.777502785315817 -2.3402777765197036, -6.815650101799636 -2.2769764112111694, -6.802330473236227 -1.4694322449864048, -5.889986578253007 -1.5996567594712578, -5.851288723821245 -2.34667315656429, -5.8749988067684304 -2.357630962197754))
+319	line_106	LINESTRING (-1.2159412728643444 4.193642589708001, -1.2515497499594024 4.992136526047356)
+320	point_106	POINT (9.176097643972916 -2.1748325578618086)
+321	poly_107	POLYGON ((-5.9418746395059125 6.63388329986039, -6.751305087054738 6.674335673952132, -6.82055182550312 6.743588010718485, -6.881884687563841 7.5504832762857, -6.750009061149352 7.6086946900217765, -5.9330971665703105 7.5201927748969535, -5.9418746395059125 6.63388329986039))
+322	line_107	LINESTRING (-3.5888689112434666 2.712651772277753, -3.6350853151713896 3.4505678131843993)
+323	point_107	POINT (-7.363314563588173 -8.441998080990597)
+324	poly_108	POLYGON ((-7.701415806359978 5.657482822973552, -8.349053282998538 5.811290014694252, -8.480972159819467 6.602456002257081, -8.400035611088326 6.617733727374586, -7.674862620918982 6.51462270060486, -7.701415806359978 5.657482822973552))
+325	line_108	LINESTRING (-1.0192892067306127 -4.8201975114534, -0.9446454035093653 -5.605490729244888)
+326	point_108	POINT (5.077919419693541 -8.91333143162943)
+327	poly_109	POLYGON ((-2.3223803517143677 -7.174210137084778, -2.353938609198506 -7.136510589588283, -1.5774886533206507 -6.267902943978661, -1.5599657742436888 -6.259398250941088, -1.4419347436789085 -6.343861607233974, -2.3223803517143677 -7.174210137084778))
+328	line_109	LINESTRING (9.702713033711797 -6.920452424457294, 9.696941089482587 -7.026495123508121)
+329	point_109	POINT (-6.5355139612800865 5.0721021550560135)
+330	poly_110	POLYGON ((7.718872426740685 -5.719355096176974, 7.689833173339578 -4.850130390041344, 8.505900176731547 -4.7728256542561995, 8.580706686995603 -5.561537591044349, 7.718872426740685 -5.719355096176974))
+331	line_110	LINESTRING (5.551810598222208 0.3114020958112468, 6.406062148153463 0.2223519201601127)
+332	point_110	POINT (5.676716868336848 5.1114292014043645)
+333	poly_111	POLYGON ((9.005734421928913 3.3730055637316454, 8.163958865346178 3.403008131182109, 8.194122065651696 4.214578125015673, 9.03887348932734 4.244229910830475, 9.005734421928913 3.3730055637316454))
+334	line_111	LINESTRING (8.038809251908395 4.611869952995944, 8.999356682915385 4.657759113481657)
+335	point_111	POINT (-3.3918314278362662 4.534778358973697)
+336	poly_112	POLYGON ((9.583358465262943 -7.079677739556352, 9.503179912391154 -6.321997957730189, 9.658620369645604 -6.216546948111971, 10.338147493952263 -6.198171728287044, 10.345343216328883 -6.273950797103475, 10.336492758742322 -7.055277768843996, 9.583358465262943 -7.079677739556352))
+337	line_112	LINESTRING (8.371757003722726 -3.068643524818022, 8.30718418445942 -3.062411229821349)
+338	point_112	POINT (5.809666229087068 -9.580899713414611)
+339	poly_113	POLYGON ((-4.831568992509331 -7.2157679093504745, -4.96607305088774 -7.1416405054687075, -4.9327347335768605 -6.339360442397226, -3.985745873393683 -6.261712670837647, -3.9867658521222418 -6.375535775248358, -4.1595122214385585 -7.2055984339701675, -4.831568992509331 -7.2157679093504745))
+340	line_113	LINESTRING (-4.187994122414326 -3.6965655097365495, -4.823878299936529 -2.8356630025589986)
+341	point_113	POINT (-3.6752590630995385 -8.730692731708867)
+342	poly_114	POLYGON ((-5.338189924992597 1.5140697931824714, -5.366260268572664 1.545562211767887, -5.383608686035972 1.5664867978011534, -5.382797213990506 2.219586567761356, -5.308434925029769 2.245946412651963, -4.533917603292482 2.2653000126834737, -4.546592208918113 2.196506578950684, -5.338189924992597 1.5140697931824714))
+343	line_114	LINESTRING (3.116021209374686 8.02346973567435, 3.8125362523240924 7.920214935256025)
+344	point_114	POINT (-2.400591494258255 -4.603743557571968)
+345	poly_115	POLYGON ((2.4600105502585814 -10.118787125548836, 2.3984749474413 -10.071233395662452, 2.362844883133441 -9.942265240403055, 2.327324520861046 -9.178245500410648, 2.3736111952552816 -9.163008846580812, 2.402264744706126 -9.168250023648893, 3.2020861377971084 -9.983746699274093, 2.4600105502585814 -10.118787125548836))
+346	line_115	LINESTRING (0.6832694875852355 -3.436467343844666, 0.5677939635306504 -3.4667461021065997)
+347	point_115	POINT (2.7253627045180466 3.5214293587047063)
+348	poly_116	POLYGON ((-1.5736858037747627 8.287787675702393, -1.7548247909056311 9.167558041638387, -1.6737971601476949 9.209520199666082, -0.8737078824927029 9.278296222077831, -0.8589026959803657 8.34576375151233, -1.5736858037747627 8.287787675702393))
+349	line_116	LINESTRING (-7.696517819903351 -4.0800860806653825, -7.740233695449238 -3.3669062080550667)
+350	point_116	POINT (3.081230045473027 0.7408341199764414)
+351	poly_117	POLYGON ((-8.656560004400346 -8.575306776120529, -9.4410113108738 -8.571359804565766, -9.522173750554202 -8.545033529020621, -9.612147138409636 -7.642363268596816, -9.531031998146144 -7.629277993323351, -8.778464958454002 -7.658186351634663, -8.656560004400346 -8.575306776120529))
+352	line_117	LINESTRING (7.977279187380359 -3.8639785448213178, 7.225037116563792 -3.7435937816380136)
+353	point_117	POINT (-4.661129270844235 2.3675891541138303)
+354	poly_118	POLYGON ((4.592764191484713 3.847699709725765, 4.5140774800144055 3.8563413393850814, 4.540889173984576 3.93528329774964, 5.352338263672417 4.639906689171719, 5.3678324531192105 4.643577619606677, 5.441862939364725 4.648013589541726, 5.400843545349846 4.553315839770259, 4.592764191484713 3.847699709725765))
+355	line_118	LINESTRING (1.222391164590115 9.903060682241339, 0.2945669866145263 9.18145678211366)
+356	point_118	POINT (1.2530918550052434 4.781766830392043)
+357	poly_119	POLYGON ((-4.184509205923159 3.150646744615217, -5.025431084434585 3.9524229947023395, -4.109128799417437 4.142318280076858, -4.1691601528860325 3.329512063976105, -4.184509205923159 3.150646744615217))
+358	line_119	LINESTRING (-2.85270181696326 1.661066240975176, -2.727489838569818 1.6666368480737326)
+359	point_119	POINT (9.631051576676073 9.394377487991203)
+360	poly_120	POLYGON ((8.186162481139862 -4.113229365402448, 7.413197337290202 -4.074738548198306, 7.294850773146548 -4.064209440492917, 7.299423626395293 -4.012833861055164, 7.421252354745475 -3.305453989485812, 8.041136053953519 -3.2828768078056076, 8.128713970457973 -3.326605592797044, 8.186162481139862 -4.113229365402448))
+361	line_120	LINESTRING (2.9698984441087126 3.035072934754279, 3.854687497274962 2.245613123791541)
+362	point_120	POINT (-5.584737580704771 8.006266271459461)
+363	poly_121	POLYGON ((-7.651762906743884 0.2335244244959194, -8.576638727325802 1.0452771407324686, -8.610393339821604 1.1666621709917089, -7.707205604095488 1.1244240849740383, -7.662413873821115 1.0407037607857395, -7.651762906743884 0.2335244244959194))
+364	line_121	LINESTRING (-6.3477839873235125 5.846996472478236, -6.28711164690371 5.167540041541359)
+365	point_121	POINT (-0.1731436120972777 5.550158624889926)
+366	poly_122	POLYGON ((-1.4772109054000953 -10.289331214768142, -1.4940852016899953 -9.485914447209494, -1.4205749457055064 -9.35535266968594, -1.3419950372241338 -9.396649159797677, -0.5838928009903942 -10.207101222754808, -1.4772109054000953 -10.289331214768142))
+367	line_122	LINESTRING (4.4834836499475195 -2.5610052786288255, 5.238706949035187 -1.8365031234087592)
+368	point_122	POINT (8.355904633408802 -2.564082785671227)
+369	poly_123	POLYGON ((-6.67725159278268 -8.819468684820492, -6.763268192478025 -8.805975039918007, -6.6467165883568535 -7.97737046050142, -5.85907444135494 -7.883048026477012, -5.9817621748112355 -8.712136586508178, -6.67725159278268 -8.819468684820492))
+370	line_123	LINESTRING (5.452737428734956 4.824535653896828, 5.517143005517474 4.121840290401042)
+371	point_123	POINT (-0.1021819344142138 -5.196326276744931)
+372	poly_124	POLYGON ((-10.18997617766545 3.1580904804364676, -10.15740193263316 3.9688011537522763, -10.065564639614788 4.077097892260775, -9.322647148040431 4.083731503065579, -9.228984237943475 3.935030200728684, -10.18997617766545 3.1580904804364676))
+373	line_124	LINESTRING (3.406103458704542 -0.7228211268266795, 3.423070134046552 -0.76084515501836)
+374	point_124	POINT (-8.675749133668333 -5.732609813841348)
+375	poly_125	POLYGON ((6.4091584469549465 2.673260925368385, 6.270004555697791 3.494809760965335, 6.401883172358122 3.6296901999069924, 7.1183643363670015 3.641681631644576, 7.242914156461646 3.507881318455393, 7.120903634173615 2.7574878234380087, 6.4091584469549465 2.673260925368385))
+376	line_125	LINESTRING (1.2772713167389984 -4.938204442116279, 1.4083358547694549 -5.6683761688775975)
+377	point_125	POINT (6.568809714201341 1.6937415284416968)
+378	poly_126	POLYGON ((-0.0206063420091533 8.681959001439921, -0.806446616359876 8.74412320090628, -0.8683679040314605 8.790667444265319, -0.7605155561863304 9.663209284138365, 0.0025975657188241 9.558149383631871, -0.0206063420091533 8.681959001439921))
+379	line_126	LINESTRING (5.9280747139755405 -8.338875490291366, 5.26820691823622 -8.34447025263347)
+380	point_126	POINT (3.98834160820157 -4.5000789019644785)
+381	poly_127	POLYGON ((7.2472335661432465 -10.316746029502871, 7.149305827960721 -9.342696094184411, 7.899681800769449 -9.380843983414366, 7.984624812122869 -9.392812127920749, 7.969859829582777 -10.208206483574354, 7.2472335661432465 -10.316746029502871))
+382	line_127	LINESTRING (-7.400220812814102 5.462805513769048, -7.464197418605849 6.190315497598509)
+383	point_127	POINT (9.53304031672451 -4.434525497334925)
+384	poly_128	POLYGON ((-4.4527414997717045 -4.409183027976552, -3.688331738073749 -3.4490274796961518, -3.56173142695909 -3.4433792224125517, -3.5116178569869967 -3.5691379113352157, -3.635472668079478 -4.394619989677864, -4.4527414997717045 -4.409183027976552))
+385	line_128	LINESTRING (4.019679647466255 3.6432662158242417, 4.083584716022963 3.5544858316810806)
+386	point_128	POINT (-4.785186383798667 -3.9935403037480297)
+387	poly_129	POLYGON ((-8.95458176728793 -2.0506102803909756, -9.002476561502508 -1.2111370331472389, -8.978789888890523 -1.1194204611079894, -8.191909174496175 -1.2429365236366046, -8.09140891551975 -1.8893451279202265, -8.95458176728793 -2.0506102803909756))
+388	line_129	LINESTRING (-1.3604587245339501 9.001979445119918, -0.7249859448848701 9.808572818192136)
+389	point_129	POINT (-7.479801606873837 -4.197306380874129)
+390	poly_130	POLYGON ((-2.833923005019807 -4.0050156675879, -3.5795835034010643 -3.1604375328323915, -3.460806898062753 -3.0318416847161607, -2.813280770274134 -3.0333187442157605, -2.7543246503329533 -3.068315565747046, -2.660729990163076 -4.002542393183679, -2.833923005019807 -4.0050156675879))
+391	line_130	LINESTRING (-3.5439381360379496 -0.3270650985484848, -2.727998802977248 -1.0925953148479)
+392	point_130	POINT (-7.931371707904061 -7.304466116085791)
+393	poly_131	POLYGON ((1.3775838897852462 -8.528595799929406, 0.6801184085473562 -8.49001521682783, 0.6569695489678995 -8.375625157709568, 0.6085165455689635 -7.6063582806414916, 0.6428296199636534 -7.561407276523766, 1.3693766163139511 -7.619242563413836, 1.4347648702451608 -8.40152405241203, 1.3775838897852462 -8.528595799929406))
+394	line_131	LINESTRING (2.7101658032498483 2.015525663185029, 2.7545135087915193 1.9667124234514122)
+395	point_131	POINT (-6.741674112601121 2.5334450335654033)
+396	poly_132	POLYGON ((5.6424674953533565 1.925454902293904, 5.61476646545249 2.033712554991127, 5.558248754807851 2.703496075724059, 5.600940625994355 2.7382559252320413, 6.544894058749373 2.8557799882893513, 6.455547644885771 2.7293981806444934, 5.6424674953533565 1.925454902293904))
+397	line_132	LINESTRING (2.6630223631465144 6.487599272060878, 2.000887327121988 5.672889049834244)
+398	point_132	POINT (1.8387425570063505 -5.389317843188907)
+399	poly_133	POLYGON ((4.3541601924989415 0.4634349946875475, 3.5555939245307737 0.5242038117281934, 3.4923266722463744 0.6144796137794062, 3.596740420531867 1.381094507289171, 3.6439871349433144 1.3819553656523056, 4.380787694173018 0.6129600191197402, 4.4109909558359615 0.524714185456935, 4.3541601924989415 0.4634349946875475))
+400	line_133	LINESTRING (7.122340917571807 1.9395441520651606, 6.31027665721111 1.9932063208024473)
+401	point_133	POINT (-9.553603605305609 7.4852061279386435)
+402	poly_134	POLYGON ((2.5200545407951256 8.35141767543691, 1.6272601975741314 8.425574167696654, 1.5924723718905325 8.47411881631676, 1.666979286427717 9.184019788993258, 2.4487946931670344 9.227128535720716, 2.5200545407951256 8.35141767543691))
+403	line_134	LINESTRING (1.9176315387127645 -0.1010531283483362, 1.0096151444134156 -0.1622290548166534)
+404	point_134	POINT (-3.5388792914053293 1.6501060309393096)
+405	poly_135	POLYGON ((0.016750747446649 5.834420982139431, 0.0798057034561567 6.664419388426918, 0.1104397704886023 6.6702828307693345, 0.8570952287157559 6.566136984002651, 0.8254253269771417 5.868272844411934, 0.016750747446649 5.834420982139431))
+406	line_135	LINESTRING (7.088044550275242 1.7023442163396731, 7.987037230329186 1.8055320830517712)
+407	point_135	POINT (-8.125235666589308 -2.3213668164527843)
+408	poly_136	POLYGON ((8.452049361286855 -6.889824857193462, 7.639572686697566 -6.8840945436839, 7.574327561428303 -6.813688124694299, 7.5936999124954845 -6.150299072350718, 8.456197402119408 -6.036869753980844, 8.465007825480596 -6.11493022398271, 8.452049361286855 -6.889824857193462))
+409	line_136	LINESTRING (-8.081515706342508 -9.550615735430632, -8.862321596785408 -10.21004998956359)
+410	point_136	POINT (2.202133193823911 -8.905618444346105)
+411	poly_137	POLYGON ((-8.332539450471087 4.001498947303802, -9.191527489850932 4.053486505996208, -9.224473213840772 4.062337732956178, -9.287050966856153 4.8244000654560235, -9.17920858953766 4.917981952922161, -8.324389187855001 4.756572604520084, -8.332539450471087 4.001498947303802))
+412	line_137	LINESTRING (4.268820840991835 9.053094464602962, 4.359519405428701 9.8384691242436)
+413	point_137	POINT (6.634410867120435 8.542382585904496)
+414	poly_138	POLYGON ((-10.42489504846914 1.58999653454479, -10.37074629106515 2.2891277986258194, -10.325984160047097 2.415477034796963, -9.617057820691729 2.372042821013425, -9.50730307831899 2.267425182925863, -10.42489504846914 1.58999653454479))
+415	line_138	LINESTRING (-1.6235404700996159 2.7698777724683423, -0.9245892967896112 2.7024270762068188)
+416	point_138	POINT (6.607630724818255 -2.8485431975286293)
+417	poly_139	POLYGON ((-9.485886913411928 -0.8723667178573048, -9.54157511715955 -0.8676377272678217, -9.553455931333206 -0.0856042765887932, -8.61855998799075 0.0723062370841936, -8.571321481580336 -0.1079496175252852, -8.667899941940394 -0.796501328471135, -9.485886913411928 -0.8723667178573048))
+418	line_139	LINESTRING (-2.7425673494358205 -9.067649639840258, -3.592913086842051 -8.299758145878599)
+419	point_139	POINT (4.001232762057041 -4.316732678682762)
+420	poly_140	POLYGON ((-4.061636093671189 0.0361573245525853, -4.213479136324768 0.0376538022738703, -4.126390453690166 0.9346030838266476, -4.10308998087617 0.9867121636844319, -3.2615819377176063 0.1757716110095147, -4.061636093671189 0.0361573245525853))
+421	line_140	LINESTRING (-5.251711520762994 -10.430527609618004, -4.330161298270706 -9.5362292675306)
+422	point_140	POINT (7.017889663588101 -9.043678943902087)
+423	poly_141	POLYGON ((3.3971894995208936 3.0413868327571523, 2.577128822720618 3.0523666126774436, 2.5379325562292365 3.924314653230089, 3.3129238339247693 3.8995913310959924, 3.3971894995208936 3.0413868327571523))
+424	line_141	LINESTRING (7.055749113763863 5.915690299446237, 7.023589921923417 5.916931297294919)
+425	point_141	POINT (6.556822842786052 -2.3350862525863523)
+426	poly_142	POLYGON ((9.309936516776386 -4.114304024289709, 8.503357792720475 -4.086899858865383, 8.468921702916768 -3.936144847399241, 8.482687137275136 -3.1268471536207625, 9.32604733764659 -3.159452623557125, 9.438004507428955 -3.9241442163537563, 9.435359805634127 -3.982729466356206, 9.309936516776386 -4.114304024289709))
+427	line_142	LINESTRING (-6.733224765748693 2.8215649145464017, -6.640744635076185 3.5609313128633966)
+428	point_142	POINT (-3.6175941322608893 4.499037142623759)
+429	poly_143	POLYGON ((6.494202324308303 -8.644092325607536, 5.723286387356447 -8.524395391624099, 5.714269288734941 -8.457169669059367, 5.662037776889829 -7.8169878757545765, 5.713382667044959 -7.697322729268063, 6.49256311884877 -7.664131285664555, 6.494202324308303 -8.644092325607536))
+430	line_143	LINESTRING (7.820217036684391 -2.258717639343024, 7.9522189070673885 -3.025427658932717)
+431	point_143	POINT (9.01358858466851 8.304933076876338)
+432	poly_144	POLYGON ((-4.347395102269583 1.1003651867518682, -5.250645549356911 1.119209667145924, -5.280536860414042 1.9556366168318047, -4.481641465887422 1.951094932759299, -4.347395102269583 1.1003651867518682))
+433	line_144	LINESTRING (-7.730201768030855 7.518830799240992, -8.535173043189525 7.577316876166643)
+434	point_144	POINT (6.072807301566113 6.654864333883757)
+435	poly_145	POLYGON ((2.2662710019536076 -7.562933128727823, 1.3336476366099677 -7.545604500188098, 1.289408549113957 -7.525022981175462, 1.397157495455672 -6.667794132810816, 2.099501789615154 -6.638172119732435, 2.2662710019536076 -7.562933128727823))
+436	line_145	LINESTRING (4.53724833198446 -2.8787291175225023, 5.315501138538855 -3.6057435350299922)
+437	point_145	POINT (4.247687414502357 9.450711456366863)
+438	poly_146	POLYGON ((1.747780972273984 -7.256184348728294, 1.7355023056711119 -7.254804706645816, 1.7226082089610342 -6.420962200085477, 1.7733953501959667 -6.367353637255549, 1.8876284304485866 -6.331759287896317, 2.70106334273628 -6.3540129429177625, 2.6285901885719305 -7.136152002635645, 1.747780972273984 -7.256184348728294))
+439	line_146	LINESTRING (-2.6868975034165277 9.27972099688642, -1.7836723077021415 8.441554665610818)
+440	point_146	POINT (2.192779874952251 -7.143361269656567)
+441	poly_147	POLYGON ((-0.1773793732144361 9.357696708928, -1.0493352415031456 9.448726990301031, -1.0847803661073943 10.197237499872763, -0.2818704695132022 10.317265404948053, -0.2412395440721528 10.302729421409223, -0.1773793732144361 9.357696708928))
+442	line_147	LINESTRING (9.158169808643907 8.15928700254269, 9.162147131013555 8.158847316242543)
+443	point_147	POINT (0.9829697450860483 -4.526444315901868)
+444	poly_148	POLYGON ((7.819464417697807 1.6505070424929893, 7.174034545496263 2.2820368139406924, 7.087076925934185 2.4533470328661524, 7.982482545391999 2.4688776529371688, 7.96672741801627 2.370117450911293, 7.819464417697807 1.6505070424929893))
+445	line_148	LINESTRING (-6.671120066822864 2.839719713447209, -6.664925635906823 2.0025285499831345)
+446	point_148	POINT (9.977763963331817 -8.788415893403732)
+447	poly_149	POLYGON ((-3.143983675485921 -8.446346119456818, -3.902267447487444 -8.355643927929249, -3.8458475257263607 -7.5484636590377185, -3.01734252719986 -7.5985784475341225, -2.984910987238134 -8.43718093691048, -3.143983675485921 -8.446346119456818))
+448	line_149	LINESTRING (-8.676800151551307 2.4224592400518077, -8.559211546288978 1.7350283535506532)
+449	point_149	POINT (-7.654636187456992 -3.7867012738051105)
+450	poly_150	POLYGON ((7.053782261742471 1.4118291651547006, 6.120089032595589 1.500667506030434, 6.223228848654924 1.5251991072482034, 6.990338453854356 1.4699619156964112, 7.02731900761652 1.4496437206252528, 7.053782261742471 1.4118291651547006))
+451	line_150	LINESTRING (8.235846690057233 4.518090556158196, 8.173900160521647 5.490859196821537)
+452	point_150	POINT (-5.382528371292748 -5.130494838959949)
+453	poly_151	POLYGON ((0.9990857389099096 6.872802161487233, 0.9627056207004829 6.953050516606574, 0.9388083397353777 7.622679291061056, 1.8006222653286237 7.770600536048912, 1.8190507003112342 6.974377167944807, 0.9990857389099096 6.872802161487233))
+454	line_151	LINESTRING (-9.662943660411742 8.15335369332908, -9.733544568977871 8.202600143122678)
+455	point_151	POINT (9.3592884994467 4.553087421593685)
+456	poly_152	POLYGON ((-6.566931263021797 8.150173360634763, -7.495135118847325 8.159279554417473, -7.49158472451769 8.999480374383849, -7.3953716100076194 9.064448481738735, -6.608698871917418 9.070301730163447, -6.559026416245212 8.258283083875206, -6.566931263021797 8.150173360634763))
+457	line_152	LINESTRING (0.0792913154747885 5.654996596913573, 0.0446465903461345 6.444423576958837)
+458	point_152	POINT (-0.7350713847154732 -0.6121527053896897)
+459	poly_153	POLYGON ((5.482912663661234 7.357600567183093, 5.471758710239371 7.493438444881507, 5.460208350145813 8.150668625232377, 6.2117783075943755 8.321931732430281, 6.342756032890259 8.250815694354717, 6.253758649650558 8.156335474559185, 5.482912663661234 7.357600567183093))
+460	line_153	LINESTRING (-0.6903844985243095 8.56045143803609, -0.6616599069500897 9.271313284281906)
+461	point_153	POINT (3.8717047303406757 3.20776060246507)
+462	poly_154	POLYGON ((9.782429986510797 3.9473434214672407, 9.10060263401013 3.9590026727816605, 9.015052088072835 4.675193879001775, 9.74250880739716 4.785496020641231, 9.881147678462234 4.666474795496323, 9.900866943785898 3.96305132507661, 9.782429986510797 3.9473434214672407))
+463	line_154	LINESTRING (3.8813173057597803 5.944056333101093, 3.157008709711848 5.985848744263546)
+464	point_154	POINT (3.312724930489501 2.0646464700295803)
+465	poly_155	POLYGON ((-9.083032540804627 -7.869494852309543, -9.124776069600427 -7.860075614662478, -9.15347960628798 -7.066497181817272, -8.279969518980073 -7.022772364904803, -9.083032540804627 -7.869494852309543))
+466	line_155	LINESTRING (-1.6383964354540437 7.680618591537496, -2.404060097437627 6.836185370547683)
+467	point_155	POINT (2.338734892820348 0.7054489143530438)
+468	poly_156	POLYGON ((7.564649577658485 6.718187316759212, 6.780461212712609 6.883791814414164, 6.789982629234658 7.587635046575326, 6.834457236221666 7.671956962617516, 7.651179467969302 7.637363687999325, 7.691135393290135 6.894228390168078, 7.667424749488308 6.806866953037533, 7.564649577658485 6.718187316759212))
+469	line_156	LINESTRING (-0.1226052350733454 -8.699460772461759, -0.267712888998986 -7.787895058199495)
+470	point_156	POINT (-3.5509958756635456 9.035700243343443)
+471	poly_157	POLYGON ((-5.761046719092256 7.851936468374703, -5.85554111712813 8.732272023273406, -5.095323119286484 8.695256612499088, -4.968029167054722 8.66712994163672, -4.952505682725519 7.936278984576996, -5.761046719092256 7.851936468374703))
+472	line_157	LINESTRING (-5.072315929803056 -7.558422713357229, -4.228779330566724 -8.359991063600681)
+473	point_157	POINT (-8.330845979308906 7.975327505387213)
+474	poly_158	POLYGON ((4.403327935452047 8.114894246349241, 3.492717809906806 8.126214555413707, 3.496139352914566 9.056705594462365, 4.372343382735897 8.899743512631584, 4.403327935452047 8.114894246349241))
+475	line_158	LINESTRING (4.876609976229354 -7.328642619687401, 4.912397178529485 -7.351416884495797)
+476	point_158	POINT (-2.4908443357083985 2.4472223511592923)
+477	poly_159	POLYGON ((-0.1122286026447474 -5.004279294797908, -0.2676451198557479 -4.145070092369212, 0.6321931151777627 -4.233666797373306, 0.6737057682701266 -4.874975279129612, 0.6485230218164827 -4.94540782643373, 0.530549398193084 -4.9642815167862615, -0.1122286026447474 -5.004279294797908))
+478	line_159	LINESTRING (-9.182282737894377 4.69595122253395, -8.290267444742039 4.724412305070384)
+479	point_159	POINT (-10.016519160483533 3.5796762996386597)
+480	poly_160	POLYGON ((-1.368525886986326 -10.394347023314694, -1.3944214220570705 -10.33764747860589, -1.3297244101412624 -9.50783488377758, -0.4776159358773651 -9.573731918210541, -0.4317127748156427 -9.641311748863618, -0.595572369564869 -10.30236223240558, -1.368525886986326 -10.394347023314694))
+481	line_160	LINESTRING (-8.417216217417163 8.797801452765286, -7.654399621614967 9.622998270739723)
+482	point_160	POINT (-4.804436160431238 8.252853307784784)
+483	poly_161	POLYGON ((-9.839394202351887 -4.549862206301896, -9.774782062569468 -3.725034120766451, -9.73089827533027 -3.6639934772306146, -9.01667088300166 -3.768857748881416, -8.872271424786426 -4.537249214732991, -9.839394202351887 -4.549862206301896))
+484	line_161	LINESTRING (5.243681714817792 -2.728579764279605, 5.406866295553956 -1.9369331688000402)
+485	point_161	POINT (5.127144066780468 -2.2304999582625205)
+486	poly_162	POLYGON ((3.4003459000943925 -5.9479601600774155, 2.628902606598473 -5.839542709709481, 2.5855634416236493 -5.007593647412613, 2.6932899935673866 -4.9548730206074785, 3.3911003882551274 -4.98442175626558, 3.4716849870366873 -5.851337890547976, 3.4003459000943925 -5.9479601600774155))
+487	line_162	LINESTRING (2.790177973012846 7.66417154898687, 3.6066838755174566 7.6012586662026305)
+488	point_162	POINT (4.343374935134628 9.290033921824715)
+489	poly_163	POLYGON ((-1.309207375084406 -1.8652231052345027, -1.245293906365754 -0.8886724889081513, -0.4274896694214458 -0.9225638136651642, -0.4285665698263342 -0.9546457053360581, -0.4627962919504344 -1.0346732385145718, -1.309207375084406 -1.8652231052345027))
+490	line_163	LINESTRING (9.104816056443394 -1.2887057439153877, 9.126340113485693 -0.361011418106808)
+491	point_163	POINT (8.706571591592596 5.996994237348767)
+492	poly_164	POLYGON ((8.413762914396235 -2.482050421839214, 7.51454624282566 -1.6375543316708154, 8.419479103449762 -1.5091190505760947, 8.452538650378814 -1.5419304231103483, 8.413762914396235 -2.482050421839214))
+493	line_164	LINESTRING (-9.847015456126726 -8.381167357727834, -9.836641758559592 -8.347964625936386)
+494	point_164	POINT (9.518104529568888 3.653003124470476)
+495	poly_165	POLYGON ((-2.378106216019865 -7.57301182934924, -3.1070728946051127 -7.49248143916599, -3.116247544832991 -6.593501850080975, -2.3334779800132615 -6.700916173114097, -2.2350337661351087 -7.505185545073761, -2.378106216019865 -7.57301182934924))
+496	line_165	LINESTRING (2.5617389568156472 -1.2875388447571074, 2.502629805423105 -0.4658890811638117)
+497	point_165	POINT (4.292919172094518 -8.461224744683305)
+498	poly_166	POLYGON ((6.593759397386073 2.537992495233442, 6.556873028162782 2.5416300039113966, 6.51028958359473 2.6036250971049664, 6.498238308182314 3.510291035671325, 6.595210553462997 3.518469368926449, 7.304406089946097 3.3913391839308096, 7.232249959466587 2.5564440846826058, 6.593759397386073 2.537992495233442))
+499	line_166	LINESTRING (-1.2186875800821413 -6.716924545340206, -1.2146808047550275 -7.596408973598961)
+500	point_166	POINT (-0.1659597291729691 -6.080605816197006)
+501	poly_167	POLYGON ((-1.2240295612921928 6.529695308678106, -2.065811411726042 6.557072972076865, -2.158503113394617 7.190476365605476, -2.0741596055584317 7.3432056838313216, -1.2305943398492722 7.3029236481562325, -1.2240295612921928 6.529695308678106))
+502	line_167	LINESTRING (-1.6823101392532147 -8.886939374685259, -1.6506974141663995 -9.70594063532547)
+503	point_167	POINT (1.2837836575563557 -8.827984725170968)
+504	poly_168	POLYGON ((5.305939790366354 0.3292651958368027, 5.268193635530309 0.354124460236627, 4.5178859247317265 1.1435454597322952, 4.565926502441175 1.2361691505095993, 5.37168546473082 0.439819047208112, 5.351937831442388 0.3466291959473375, 5.331052703945084 0.3321632652134895, 5.305939790366354 0.3292651958368027))
+505	line_168	LINESTRING (8.022118392173677 9.425293947269148, 8.698481801432795 9.603035885770543)
+506	point_168	POINT (1.567242593621721 10.393487614629406)
+507	poly_169	POLYGON ((-7.567374390667216 -9.537192474354923, -8.461284471648101 -9.472587503270493, -8.418222046789577 -8.556687122002353, -7.517538774477207 -8.578818145518136, -7.495751571960079 -9.353994220898569, -7.567374390667216 -9.537192474354923))
+508	line_169	LINESTRING (5.70246440856428 -7.315480345014678, 5.784403949765098 -7.287029057851084)
+509	point_169	POINT (-4.239358550110509 10.120216600195192)
+510	poly_170	POLYGON ((-8.042805547300492 -8.591396212970528, -8.12866386354558 -8.44033706051302, -7.2920629353774995 -7.682476393993319, -7.211902531529423 -7.698667583324366, -7.200454234954529 -8.58615313954071, -8.042805547300492 -8.591396212970528))
+511	line_170	LINESTRING (-9.05756502201002 -1.0874566605804743, -9.786287431295962 -1.1500359291436835)
+512	point_170	POINT (1.9025880770687136 7.085602519624324)
+513	poly_171	POLYGON ((3.2066051030881537 6.372436578255002, 3.125517100265039 7.254819892692842, 4.020851440429091 7.291422625411558, 4.026801502319269 7.2170306076322746, 3.922333250198963 6.37573733013505, 3.2066051030881537 6.372436578255002))
+514	line_171	LINESTRING (6.692283516024988 1.016935850589005, 6.669265714693038 1.7973484043332153)
+515	point_171	POINT (-2.916648338808264 -5.58659157687378)
+516	poly_172	POLYGON ((5.281436876819919 -1.026270373258856, 4.540601828562627 -0.9970476120872047, 4.53496845318815 -0.9628784436589336, 4.518987820289383 -0.8612872651420784, 4.430075149160295 -0.0470849347530442, 4.501053805756435 -0.0427468723600887, 5.215626254787482 -0.9358676329133386, 5.281436876819919 -1.026270373258856))
+517	line_172	LINESTRING (4.1355004602874414 7.85312576099452, 3.9612308418284847 7.156508913858831)
+518	point_172	POINT (4.835927637307697 -0.6301148281144178)
+519	poly_173	POLYGON ((-2.4138848878385635 -7.55970099668986, -2.4379142648243586 -7.418911258906266, -2.4858188703540955 -6.709603288037774, -2.4376230287154224 -6.621282318056479, -2.393830806058414 -6.580273872639316, -1.6087669220855156 -6.645035330717208, -2.4138848878385635 -7.55970099668986))
+520	line_173	LINESTRING (4.222666418374664 -7.743291667839514, 4.1058285084408 -7.871905084770512)
+521	point_173	POINT (-0.7461079489014759 5.413812032358231)
+522	poly_174	POLYGON ((2.5706251271202256 -9.896377894039754, 1.677908425764726 -9.852259389942054, 1.6242091579032976 -9.034120074940532, 2.5176383538950056 -8.947461236707518, 2.5915837389261935 -9.71351401921326, 2.5706251271202256 -9.896377894039754))
+523	line_174	LINESTRING (-7.050402237875272 1.1455267975967307, -6.951612733197967 0.3593934709784715)
+524	point_174	POINT (-8.766422481878898 -9.024618146851413)
+525	poly_175	POLYGON ((-0.3002256160963811 -2.5285809613780956, -0.3366600939600033 -2.433376122377441, -0.4265969479944238 -1.6203153347629422, 0.4831624266518453 -1.7099795023923583, -0.3002256160963811 -2.5285809613780956))
+526	line_175	LINESTRING (6.721512408440797 4.6439900834498, 7.45563247564799 4.697212425008959)
+527	point_175	POINT (-2.4407520591575373 2.3391884457902874)
+528	poly_176	POLYGON ((-0.9848998873479256 -3.7714360633373785, -0.9888470365168095 -3.664132976162441, -0.9988182084957453 -2.910916197501574, -0.2148598036407577 -2.8637553562777525, -0.1306000937865063 -2.9048330909916094, -0.1559490404917019 -2.9340447929533817, -0.9848998873479256 -3.7714360633373785))
+529	line_176	LINESTRING (-3.263961386489854 3.0684038739616435, -3.2417376951599786 3.0554139178713973)
+530	point_176	POINT (-5.285437815313683 -10.139844624370896)
+531	poly_177	POLYGON ((-1.6285249003968063 7.723608231149649, -2.3888075173920464 7.740151048823849, -2.281922335584765 8.63419158342711, -1.6475498635500296 8.653226724596358, -1.5477976602581087 8.6153804685339, -1.5458445976857744 7.792689288197165, -1.6285249003968063 7.723608231149649))
+532	line_177	LINESTRING (-1.684550307919957 5.037658264027069, -0.8321259133691844 4.235978275653287)
+533	point_177	POINT (9.505430521013853 -2.5876403498409335)
+534	poly_178	POLYGON ((-0.9608214501585441 0.9441732122111554, -1.782171490229326 1.0454568640573518, -1.8643010272442047 1.7128150934477406, -1.0491112269315832 1.893959647835455, -0.9755001822918002 1.802809285536088, -0.9608214501585441 0.9441732122111554))
+535	line_178	LINESTRING (0.7755368904115783 -7.6718412386405435, 1.670134872317555 -7.64565176634934)
+536	point_178	POINT (2.9256804841031228 9.662593792314226)
+537	poly_179	POLYGON ((0.996444286919595 1.7679257367596133, 0.2445016895009807 1.7709042551759107, 0.1646532651465473 2.487644703128076, 0.2498406759865448 2.564509668796468, 0.9309749715681871 2.462460843363302, 1.0534498192632324 2.3933949036732756, 0.996444286919595 1.7679257367596133))
+538	line_179	LINESTRING (9.588645878105304 7.514637631436188, 10.286928827292398 6.630141148290944)
+539	point_179	POINT (0.8760064412100512 -1.6127768095448705)
+540	poly_180	POLYGON ((0.2415064920143077 -6.459055965869022, -0.5273757163105364 -6.443990350755802, -0.6300469307178089 -6.421897355376121, -0.6124844531372604 -5.605419946290321, 0.2786030577469979 -5.584652117246184, 0.2858446838821866 -6.387701965824829, 0.2415064920143077 -6.459055965869022))
+541	line_180	LINESTRING (7.561689176410449 -1.8581779908962814, 8.347868191380892 -1.1437457169085137)
+542	point_180	POINT (-3.6971406786334655 6.6684480306859655)
+543	poly_181	POLYGON ((-7.488054368137363 -0.9780488669900416, -8.31336473622071 -0.9723701748074642, -8.325904775865336 -0.851687521393707, -8.150181090714744 -0.1101802158552468, -7.532935712608734 -0.8527263399038061, -7.516418000008587 -0.8881745728996184, -7.488054368137363 -0.9780488669900416))
+544	line_181	LINESTRING (6.752968340066886 0.9893371014798731, 7.451816603031907 1.0274125993067726)
+545	point_181	POINT (-3.145858405967087 5.3218736663085995)
+546	poly_182	POLYGON ((6.546302640150095 -7.014208754203439, 5.720381873073976 -6.9449033907242415, 5.717937039890589 -6.052926698476546, 6.57722189111798 -6.048575854776125, 6.653087217144255 -6.1432392567438585, 6.546302640150095 -7.014208754203439))
+547	line_182	LINESTRING (-5.812474130066254 1.8337011109708246, -5.8376774785190095 1.9241181434659222)
+548	point_182	POINT (7.614325430933321 3.6706567727652364)
+549	poly_183	POLYGON ((8.023188932815547 -6.973126125585026, 7.965357856862364 -6.921754997376474, 7.927337837380991 -6.231824691594952, 8.005957640343667 -6.082801068149697, 8.037393118037144 -6.135059645974875, 8.023188932815547 -6.973126125585026))
+550	line_183	LINESTRING (-5.69035848009879 -3.7259838229625264, -4.912431453008937 -3.8109476572691254)
+551	point_183	POINT (-5.498653207443628 -8.879100616883465)
+552	poly_184	POLYGON ((-3.379220346683521 4.936708401407877, -3.4084038889144783 5.820951434293399, -2.4689207227256045 5.814664238380416, -2.5290275774485367 4.986784574066333, -3.379220346683521 4.936708401407877))
+553	line_184	LINESTRING (-7.578025800361194 6.128195031906204, -6.715749977332789 6.897416995679372)
+554	point_184	POINT (8.358864534164898 -6.73507718886055)
+555	poly_185	POLYGON ((8.64586754719567 0.9260662714773452, 7.860603297518076 1.0154535136574792, 7.848710783450708 1.0909676692071153, 7.874703475529405 1.8186133836619658, 8.69081136127149 1.8300324803363435, 8.690600688211246 1.8242235189913774, 8.64586754719567 0.9260662714773452))
+556	line_185	LINESTRING (-2.4616894627850763 -2.9125227854644975, -2.3409539823481307 -2.2163644748152542)
+557	point_185	POINT (-1.9307043640802208 -1.0593780007963796)
+558	poly_186	POLYGON ((-6.519959096143134 2.804282723933962, -6.524204164058017 2.812025315184361, -6.442246058503082 2.9694531453350312, -5.704705547008472 3.687007527178313, -5.659055849032342 3.723573718789728, -5.711201598187645 2.9696389605944242, -5.731515516185043 2.8767885334537717, -5.737040532448499 2.8617964097750477, -6.519959096143134 2.804282723933962))
+559	line_186	LINESTRING (-2.659900640082172 -6.414879264555069, -2.730871425933288 -5.652552011668079)
+560	point_186	POINT (-8.575862950102785 -8.057745149975203)
+561	poly_187	POLYGON ((-4.155125510388868 6.191673193884747, -4.268887647438271 6.240197313104115, -5.086295707913973 6.986366524038439, -4.998870171681765 7.128055634841207, -4.914209135895907 7.149428117295932, -4.235042257018787 7.14296955474768, -4.11842960747344 6.328227378641177, -4.155125510388868 6.191673193884747))
+562	line_187	LINESTRING (-7.150172461771637 -5.934330242987842, -8.043683793612102 -5.993551390668563)
+563	point_187	POINT (-6.822776455518847 0.5798316111203018)
+564	poly_188	POLYGON ((-2.6561209265288515 -0.3894502643017718, -2.6554918984752582 -0.2393431615704534, -2.5512735669031352 0.5029261967022383, -1.776375234355656 0.5441820858849388, -1.7755061940618369 -0.2902389268477199, -2.6561209265288515 -0.3894502643017718))
+565	line_188	LINESTRING (1.691024259346507 -5.164898019109875, 2.4781460034391385 -4.285450986725716)
+566	point_188	POINT (4.93772315224002 6.023468762734735)
+567	poly_189	POLYGON ((4.298487959617737 1.8830891844456743, 4.368533232222539 1.970826018931836, 5.105238870758189 2.759396670715394, 5.168897865061593 1.909434212371912, 4.298487959617737 1.8830891844456743))
+568	line_189	LINESTRING (0.7966132417629553 4.900542949166139, -0.0843451984252806 4.836647733151638)
+569	point_189	POINT (8.349828619666988 7.235759352305632)
+570	poly_190	POLYGON ((-6.803398592759411 -2.5087931696603407, -6.804030340383088 -1.5603135523826661, -6.662103726684307 -1.6906648158874755, -5.817044562615499 -2.4960623118122003, -6.803398592759411 -2.5087931696603407))
+571	line_190	LINESTRING (6.354146450827551 6.837754501501816, 6.444067043526655 5.905896525916082)
+572	point_190	POINT (1.7823967737954058 4.265680129838735)
+573	poly_191	POLYGON ((-1.618246870163029 -9.76721997456089, -1.763162156046377 -9.735800116661421, -1.722421244685463 -8.851661248185888, -0.8782663420908625 -9.688476081611588, -1.618246870163029 -9.76721997456089))
+574	line_191	LINESTRING (3.4344340790971826 -6.381536184058057, 2.7279884906691065 -7.169193173265037)
+575	point_191	POINT (-1.1629440856456317 -3.2658321583016665)
+576	poly_192	POLYGON ((9.409579867857422 0.8416926602738651, 9.394321443657436 0.9890311622254653, 9.489823944866343 1.7483818000745595, 10.158378641089726 1.64727360933891, 10.160330254300522 0.9612311711561385, 9.409579867857422 0.8416926602738651))
+577	line_192	LINESTRING (7.4942852346428905 9.241177045624903, 8.214014911048821 8.42789331228779)
+578	point_192	POINT (10.145925086151504 9.095897673261225)
+579	poly_193	POLYGON ((-6.073271349339486 -6.283742631588967, -6.845878089860384 -6.211478033856931, -6.893455789224904 -6.18796590215028, -7.000235433240135 -5.534349118244191, -6.975333312463788 -5.383532791930584, -6.079150264507589 -5.473155587897946, -6.073271349339486 -6.283742631588967))
+580	line_193	LINESTRING (-2.995578270888113 -2.909094531017702, -3.045350966580011 -2.9026446528652)
+581	point_193	POINT (-1.7964465404413987 8.81791680124369)
+582	poly_194	POLYGON ((-3.3017295276469696 8.600806905120086, -2.4382619965206733 9.547866806450543, -2.4430930997103877 9.494763452955635, -2.5154767249629555 8.706820849302138, -2.600124316265447 8.650412243907306, -3.3017295276469696 8.600806905120086))
+583	line_194	LINESTRING (9.247560020253829 -1.2796653678048686, 9.419691284090705 -1.2173915518947869)
+584	point_194	POINT (5.2187789476444 2.9637548473364785)
+585	poly_195	POLYGON ((4.896281686568926 5.796012838022747, 5.026641850215685 6.596652333349175, 5.848383701578421 6.663296758522884, 5.854885183203392 5.816582429950832, 4.896281686568926 5.796012838022747))
+586	line_195	LINESTRING (-1.3187547661610124 5.763303575126252, -1.4469241972805476 5.792139481436451)
+587	point_195	POINT (-6.026785488638004 4.825803476712847)
+588	poly_196	POLYGON ((-8.59389576525299 9.423176813415783, -8.754591863564762 9.558313113479468, -8.678186738113483 10.300218624676026, -8.636365446932817 10.338089901204974, -7.931256057195982 10.286267652354853, -7.766745195809631 9.50073740776588, -8.59389576525299 9.423176813415783))
+589	line_196	LINESTRING (-8.687908477062228 2.655366194568881, -9.3936332628359 2.799078945360645)
+590	point_196	POINT (0.73040425528644 -1.2677959287234088)
+591	poly_197	POLYGON ((1.3962701908072592 4.012075445716183, 1.3415506045525603 4.025694604703387, 0.5333292545921189 4.810181896566253, 0.5914866954769528 4.8873825059393745, 1.2915201818973974 4.878651992220712, 1.3823422547730222 4.849953730000717, 1.3962701908072592 4.012075445716183))
+592	line_197	LINESTRING (2.4917319372782583 -7.338565085709147, 1.5632639008380025 -7.317307355164495)
+593	point_197	POINT (-4.253309032273918 3.9177868382161716)
+594	poly_198	POLYGON ((-8.140177375777668 -9.826186967281409, -8.979582207926063 -9.813652003105677, -9.005618272085165 -9.67449493124934, -9.077961575490908 -8.91700590597125, -9.023927785574104 -8.86935477893501, -8.157937512138407 -8.89351698459966, -8.140177375777668 -9.826186967281409))
+595	line_198	LINESTRING (-4.48643875066891 -3.8555272999286814, -5.245167773154671 -2.9527007098271874)
+596	point_198	POINT (-4.57720404258938 1.1465186508916938)
+597	poly_199	POLYGON ((-5.36599053199549 -8.062842950524429, -6.184673423001333 -8.014353566058533, -6.1395421031601565 -7.203646723587505, -5.3480406863175025 -7.134295293926495, -5.287331355169799 -7.947619053637308, -5.36599053199549 -8.062842950524429))
+598	line_199	LINESTRING (6.028787029835575 -2.7914596838369254, 5.235348884221958 -2.8287280059569593)
+599	point_199	POINT (-6.840595690493752 1.1894657610697321)
+600	poly_200	POLYGON ((-9.867050753691364 0.1922031483877678, -9.95965144567288 0.2305342635945263, -10.00719183661242 0.9033518863247948, -9.888399000172184 1.076218173274168, -9.020651021214087 1.0915872722941264, -9.106826627057568 0.9043960469894259, -9.867050753691364 0.1922031483877678))
+601	line_200	LINESTRING (1.1468123981203577 -8.555616098205435, 0.2511276414558322 -8.630506400740925)
+602	point_200	POINT (-0.905830152896382 -0.8702926213896462)
+603	poly_201	POLYGON ((-3.361272105571558 -6.850941007861022, -3.4622714181726146 -6.789507431975722, -3.45645566533868 -5.9049934505706725, -2.6305565834116336 -6.000958585407208, -2.6705033133291187 -6.726486585546578, -3.361272105571558 -6.850941007861022))
+604	line_201	LINESTRING (5.6513670559476195 -2.0864377735763977, 5.658373284119116 -1.3866219551024972)
+605	point_201	POINT (4.042781580622348 5.680953378776424)
+606	poly_202	POLYGON ((-4.086432095735628 2.68040699487224, -4.086971783243561 2.7464676836794344, -3.998846160853633 3.5981935738039716, -3.2535260007207856 3.6707130444825724, -3.105568423324308 3.608925485152062, -4.086432095735628 2.68040699487224))
+607	line_202	LINESTRING (5.870060094571432 9.808933762604314, 4.9829889023146166 9.831695579336477)
+608	point_202	POINT (-6.249850069643188 -6.2572497558837386)
+609	poly_203	POLYGON ((-0.9578318242829067 -2.812111464673438, -1.0378614155920363 -2.037873848128348, -1.0149943192892326 -1.9832904795262656, -0.9467770076017049 -1.9620443170428412, -0.2566942528182353 -2.7211481627100205, -0.9578318242829067 -2.812111464673438))
+610	line_203	LINESTRING (10.30276542832228 0.3278013817039759, 10.290789331391746 0.3119587166349016)
+611	point_203	POINT (-7.033592153008148 -4.209174254937488)
+612	poly_204	POLYGON ((5.091005478281335 -9.584474331682664, 5.161027800741893 -8.70552057661747, 5.258419601466581 -8.697067660763484, 5.898372450257056 -8.818613256121843, 5.992833977305655 -8.853112363171062, 5.913099257326083 -9.507915820258699, 5.091005478281335 -9.584474331682664))
+613	line_204	LINESTRING (-7.024988209494151 9.543375805127088, -7.837978863365502 9.441723565723521)
+614	point_204	POINT (-5.25916724417112 -8.218914663430919)
+615	poly_205	POLYGON ((0.3446050659703204 5.3087365412085745, 0.3845750659418414 6.04638390818634, 0.4960968487702722 6.119176981398648, 1.1335283884174647 6.1796864114483565, 1.2431801738647366 6.131148038380786, 1.2516413426810962 6.112439748358931, 1.1277506525881562 5.322094255790271, 0.3446050659703204 5.3087365412085745))
+616	line_205	LINESTRING (5.654419957641315 -5.09790057734983, 5.695541506613077 -5.085826675197045)
+617	point_205	POINT (-4.0614532497727405 -0.7436999449230617)
+618	poly_206	POLYGON ((1.0051962662890146 0.769618487944613, 0.8484257624432339 0.7842730230687145, 0.916036192135922 1.707940455716578, 1.7330303785057575 1.634412339886693, 1.7844470473793255 0.8681604942746095, 1.7217133315752526 0.8439510203224455, 1.0051962662890146 0.769618487944613))
+619	line_206	LINESTRING (-7.888009647121847 3.0723920332013304, -8.54657075101607 3.01563425073022)
+620	point_206	POINT (-9.456660585745322 2.028069227279955)
+621	poly_207	POLYGON ((-5.268023739643766 -2.521506589678382, -6.209956013578275 -2.410634312473975, -6.161419121263086 -1.654454939443137, -6.07729743056534 -1.623371959468105, -5.268023739643766 -2.521506589678382))
+622	line_207	LINESTRING (-1.5242310462969975 9.614611525877065, -1.5792575865009217 9.468170934495175)
+623	point_207	POINT (6.647308462933456 0.6468332394361915)
+624	poly_208	POLYGON ((8.016341245036262 1.735628233778371, 7.240824428721627 1.7529805831390983, 7.153001369182235 1.863513911244541, 7.195664139751287 2.5474282753827544, 8.054261824489382 2.652281182409999, 8.016341245036262 1.735628233778371))
+625	line_208	LINESTRING (-7.933927539493723 5.93742188583619, -8.003501954882148 5.293878008807172)
+626	point_208	POINT (-7.518438344647613 -8.577865243320396)
+627	poly_209	POLYGON ((8.046381940097781 -4.923856379655079, 7.942996413379161 -4.818950741970054, 7.8760363034348 -3.9875927124841035, 8.781097835535846 -3.9896240692415086, 8.800703940176128 -4.015118973667844, 8.84980215699154 -4.753774395054354, 8.046381940097781 -4.923856379655079))
+628	line_209	LINESTRING (7.937031125448636 1.951676767021258, 8.722682645171398 2.755866615739682)
+629	point_209	POINT (1.1878072024106867 7.829098114825426)
+630	poly_210	POLYGON ((-1.3926505182181317 6.0855903817062735, -2.143497536235241 6.091819539788051, -2.1968492008141283 7.071691101210012, -2.0435813334935653 7.081889418906773, -1.3561369635992624 6.8869084925086375, -1.234453803254919 6.158075664044996, -1.3926505182181317 6.0855903817062735))
+631	line_210	LINESTRING (-5.445033217687872 -9.61391038753053, -4.805151067699337 -9.772226394660288)
+632	point_210	POINT (8.067798520398414 8.052559785609953)
+633	poly_211	POLYGON ((0.3638821629412776 -9.989416336692004, 0.2840184737284486 -9.888884468897471, 0.2668395389125958 -9.091117595799286, 1.0641092141848958 -9.09069497898696, 1.2242195237948494 -9.953206032718171, 1.0769395752884692 -9.980901137757822, 0.3638821629412776 -9.989416336692004))
+634	line_211	LINESTRING (-7.7759577498827195 -3.739157747316534, -7.023651290327263 -3.6561903620964937)
+635	point_211	POINT (9.13543934711507 4.089796467643843)
+636	poly_212	POLYGON ((3.2267836566807597 9.46827762310253, 3.0897908843518094 10.348118363340294, 3.1484088380042596 10.427694298684695, 3.9348962547182134 10.381261048249247, 4.0443696595739205 10.322212729434534, 4.0528364006026525 9.629873867778398, 3.2267836566807597 9.46827762310253))
+637	line_212	LINESTRING (-2.838782711300566 1.6342014103097173, -3.633699770487491 1.5923078205510213)
+638	point_212	POINT (6.086325435873969 1.1339912688911926)
+639	poly_213	POLYGON ((-8.041847237086113 -8.165322983415644, -8.065942684651207 -7.2741793383009785, -7.275680063903811 -7.273684015640287, -7.207259559232432 -7.359377057051402, -7.211653181507301 -7.427059969993951, -7.975040682688267 -8.142530773020127, -8.041847237086113 -8.165322983415644))
+640	line_213	LINESTRING (7.545061655568464 -9.118401402853795, 7.440573739451757 -9.088734574498785)
+641	point_213	POINT (-5.744754469873964 -8.981743372441422)
+642	poly_214	POLYGON ((3.9794209047182214 -8.994849464700335, 3.2430848542384783 -8.981135474138291, 3.126263128402356 -8.895070589152526, 3.121301612382587 -8.194524541190006, 3.1228923649374005 -8.165297169777133, 3.194560431043056 -8.140708192362638, 4.094360890452899 -8.960515386348797, 3.9794209047182214 -8.994849464700335))
+643	line_214	LINESTRING (4.320865460880463 1.755136251755023, 5.213523798662335 2.751673489067253)
+644	point_214	POINT (-8.681529474781165 -8.213703327941534)
+645	poly_215	POLYGON ((4.630445944638731 8.303641078823095, 3.9108717098251775 8.323701610377604, 3.7960998619173836 8.383399048376754, 3.82818632732652 9.190605948999854, 4.617791469878557 9.013362890316847, 4.630445944638731 8.303641078823095))
+646	line_215	LINESTRING (1.7738169350343067 8.387942106754595, 2.587589752565245 9.252415476457273)
+647	point_215	POINT (8.472419099811178 8.217169017630372)
+648	poly_216	POLYGON ((5.137504851545851 -4.397517666246305, 4.96857001253916 -3.6496862300764388, 5.064175124459832 -3.583988404913131, 5.886939161707474 -3.559574496940275, 5.948773577617412 -3.6707980715235196, 5.833980157885121 -4.286021660123349, 5.752004296548126 -4.305589812377093, 5.137504851545851 -4.397517666246305))
+649	line_216	LINESTRING (8.634754384904157 -0.3318144990867545, 7.809878983335822 -0.2248296537157964)
+650	point_216	POINT (-6.546034738073937 4.613078792630236)
+651	poly_217	POLYGON ((-1.7504806205823942 0.9420463706046598, -2.6775819080918195 1.0208296681373636, -2.4895420815285405 1.7828623953505551, -1.8643288599196124 1.825170920819661, -1.708411644886613 1.6809231619370935, -1.7478994092201439 0.9448365270965476, -1.7504806205823942 0.9420463706046598))
+652	line_217	LINESTRING (-0.0078334290175018 -5.037705072621626, -0.7831511707289142 -5.7713430494879185)
+653	point_217	POINT (-8.17722953991313 -4.057687458292844)
+654	poly_218	POLYGON ((-3.3261058512611448 -9.449999286212288, -3.2535381849133183 -8.513693005299011, -2.4856414599951053 -8.675793400690182, -2.466393681089767 -9.391684735798787, -3.3261058512611448 -9.449999286212288))
+655	line_218	LINESTRING (-3.887714855271955 5.17443722995896, -3.903418444009844 5.294917024750737)
+656	point_218	POINT (-2.59737302520266 9.089291144981788)
+657	poly_219	POLYGON ((1.4880546937831585 -5.9241194486289155, 1.4104190132731613 -5.8626860666112615, 1.4059612075122483 -5.035621084539856, 1.4583871017457224 -4.955795446911694, 2.3383888037344382 -5.765984585706758, 2.196937757798453 -5.81109343060957, 1.4880546937831585 -5.9241194486289155))
+658	line_219	LINESTRING (1.1782025510043574 -7.97525022042197, 0.4518086974089341 -8.686770200328908)
+659	point_219	POINT (10.310141089538769 -8.466011889508891)
+660	poly_220	POLYGON ((4.284992977578209 0.3816068431258988, 4.260935481544958 0.4606266278977547, 4.275628238481199 1.1732799227958837, 5.119356197786022 1.3401397261104053, 5.021138539523089 0.4680990054330053, 4.284992977578209 0.3816068431258988))
+661	line_220	LINESTRING (-6.827666238177423 7.364515025309031, -6.985812769133527 7.312699618033661)
+662	point_220	POINT (9.590201805554546 2.762237203526182)
+663	poly_221	POLYGON ((-7.919397944697107 7.709932942141664, -8.683050141969877 7.728249769933402, -8.686162244401403 8.515958985801259, -7.76362704256408 8.687050646859033, -7.747716470349726 7.894507806861049, -7.919397944697107 7.709932942141664))
+664	line_221	LINESTRING (8.123345705985258 -2.2858168101928724, 8.109090022758648 -2.2145807178703354)
+665	point_221	POINT (-6.545384484136338 -2.5088468175047884)
+666	poly_222	POLYGON ((-6.594548048745944 -8.633145408634682, -6.714174670828184 -7.755600865517747, -5.922279732900286 -7.733805340121245, -5.841429201621287 -7.751921071214987, -5.976436635598465 -8.566201927594678, -6.594548048745944 -8.633145408634682))
+667	line_222	LINESTRING (8.548506756861988 7.435915815829634, 8.603213435213824 7.427311292441786)
+668	point_222	POINT (-6.226793119252657 6.418001865874787)
+669	poly_223	POLYGON ((-5.315740968346277 -8.008410538740247, -6.180314729993097 -7.924001116678428, -6.090148801859256 -7.106379753280177, -5.233768315177615 -7.1258062030211935, -5.2650845462347275 -7.9445021599188, -5.315740968346277 -8.008410538740247))
+670	line_223	LINESTRING (1.0809219464127824 5.611831866385656, 0.9859631381082081 6.295933624025046)
+671	point_223	POINT (6.914170612289725 -0.4107039374173796)
+672	poly_224	POLYGON ((6.832586400221059 -0.4973294928220556, 6.8249681845513726 0.3818542294011085, 7.515335589591217 0.3788674550424013, 7.66999892106319 0.2753818906189414, 7.623377268896128 -0.420424672098449, 7.5285701882827585 -0.4743102489676059, 6.832586400221059 -0.4973294928220556))
+673	line_224	LINESTRING (4.339923057883813 -5.454313839541898, 5.008969313778348 -6.322148381469341)
+674	point_224	POINT (-3.767093531391038 3.892564199537666)
+675	poly_225	POLYGON ((-8.256531195883827 0.9397775474860806, -8.988413085463774 1.107806006344744, -8.98844614789275 1.8462257476474062, -8.201162256040723 1.7810252832239493, -8.125376950684682 1.7681284326391176, -8.099219673463914 1.1178228034018627, -8.256531195883827 0.9397775474860806))
+676	line_225	LINESTRING (4.485665492319571 -8.372425891170037, 4.5517361185478205 -8.259415739730194)
+677	point_225	POINT (8.081163869249705 5.563032936131776)
+678	poly_226	POLYGON ((9.155289217128473 -3.988176293329029, 8.334787436638274 -3.9681123023543794, 8.295277927039525 -3.076471868753173, 9.15734940225715 -3.118232443081217, 9.197744960497129 -3.219458977048726, 9.155289217128473 -3.988176293329029))
+679	line_226	LINESTRING (-6.0554694212286115 -3.791985152870782, -6.83336451567991 -4.626054833646059)
+680	point_226	POINT (9.076030624440124 -8.938030468736526)
+681	poly_227	POLYGON ((1.0388159275359248 -2.861881239415673, 0.9025789785060084 -2.8459690014152565, 0.8879132686300092 -1.8965831289738764, 1.8230800855600053 -1.9821754062551205, 1.8341709764828926 -2.0581826640571883, 1.7290617313246555 -2.720391518513874, 1.0388159275359248 -2.861881239415673))
+682	line_227	LINESTRING (8.4240845778488 -0.876275103220514, 9.178883765206912 -0.9419811701707786)
+683	point_227	POINT (1.5465849343626108 -6.871147285890925)
+684	poly_228	POLYGON ((6.709510542408315 -2.938575333784331, 5.98028841595698 -2.8721896134120852, 6.007893146726358 -2.0946021734477105, 6.845414384975974 -2.0273325796577275, 6.836301968610826 -2.9292817444877035, 6.709510542408315 -2.938575333784331))
+685	line_228	LINESTRING (9.090105252184276 -4.660602000820514, 9.088670783929453 -5.41890553003873)
+686	point_228	POINT (-7.140968685354418 -4.881707697292208)
+687	poly_229	POLYGON ((3.8596440214407397 -0.629759940290766, 3.8122157632282043 0.2477557712819269, 3.9513140985371584 0.3377087836416597, 4.655635782984755 0.2319487039020155, 4.697632566777608 0.1973525399456926, 3.8596440214407397 -0.629759940290766))
+688	line_229	LINESTRING (-10.31152661857352 3.0928066255486066, -9.582490889290453 3.882475809571651)
+689	point_229	POINT (0.523762202256033 -1.5463371161177366)
+690	poly_230	POLYGON ((8.367795801732797 1.918763159352825, 8.315204784966303 1.942379441349296, 7.5623090642745385 2.846690497796325, 8.50146470309333 2.85297836228329, 8.447028188199607 2.0786759573087146, 8.367795801732797 1.918763159352825))
+691	line_230	LINESTRING (2.597483592746858 7.909288534798308, 1.7222520525243457 8.813043661225636)
+692	point_230	POINT (3.3744497221181255 -7.845633836743)
+693	poly_231	POLYGON ((6.587655960191961 -7.056026740617425, 5.732191608692627 -6.9827356303398425, 5.735370355818228 -6.121992292298096, 6.452520018453464 -6.140938550793941, 6.5870558821965846 -6.299604743838173, 6.587655960191961 -7.056026740617425))
+694	line_231	LINESTRING (-7.5890828176016045 -3.248847475400378, -7.5063823815177795 -2.443076198110585)
+695	point_231	POINT (6.891017158448956 4.314929212925785)
+696	poly_232	POLYGON ((-3.489394734225659 -5.163620330659007, -4.339187797064091 -5.161999309905433, -4.476323762829175 -5.081038050213159, -4.410236365306888 -4.381194791783707, -4.3447257653906775 -4.329570458721895, -3.5482519035662823 -4.269568867505415, -3.539776523432446 -4.274801480204576, -3.489394734225659 -5.163620330659007))
+697	line_232	LINESTRING (9.479734932527624 -7.389972950496362, 9.371521617323141 -7.357304965684283)
+698	point_232	POINT (-4.064716546550476 0.859579649630736)
+699	poly_233	POLYGON ((7.024862442543529 -8.797824270850526, 7.883740757712348 -7.953347455158922, 7.955801083311913 -7.940482986828637, 7.892701503547451 -8.70265805833723, 7.824443217405001 -8.786244681679234, 7.024862442543529 -8.797824270850526))
+700	line_233	LINESTRING (4.0610214294415865 5.817839118360737, 4.755742873358529 5.948514019842885)
+701	point_233	POINT (-6.8879668185661265 6.281729781961741)
+702	poly_234	POLYGON ((3.2736832980898707 -8.785123041783613, 2.6088842236477605 -8.776569362479153, 2.490142158144433 -8.684110338889337, 2.475529696702999 -7.938033265605854, 3.27266839563234 -7.863288933308468, 3.361373698948806 -7.994391018220377, 3.3954792129820426 -8.719790668623215, 3.2736832980898707 -8.785123041783613))
+703	line_234	LINESTRING (6.894845340810626 4.8141991023389625, 6.066525334467062 4.830738570663149)
+704	point_234	POINT (3.5804965493357166 -2.9449586485446595)
+705	poly_235	POLYGON ((1.952043332028823 -0.7748921667308017, 1.0951727897227936 -0.7096282897709179, 1.0688220547751541 -0.0048177691998942, 1.1571353570972351 0.0828925942447443, 1.986655380470625 0.1353211651286511, 1.952043332028823 -0.7748921667308017))
+706	line_235	LINESTRING (-4.043865064203233 -3.935200621983504, -4.826121479599405 -3.9560066577955326)
+707	point_235	POINT (-7.311683737335776 5.355752434787403)
+708	poly_236	POLYGON ((6.755218594557173 7.957910259786272, 6.749917697147023 7.98196624031751, 6.828521149607858 8.834276804601588, 6.83752370161592 8.869572575898884, 7.564174442066262 8.833093288089424, 7.621013980893276 8.001069407431888, 7.55227292816479 7.983399382525116, 6.755218594557173 7.957910259786272))
+709	line_236	LINESTRING (6.785722434836694 8.068895163946307, 7.469952194179282 7.269636153410958)
+710	point_236	POINT (-7.097407760685179 -9.655357901165543)
+711	poly_237	POLYGON ((-7.870735107331853 6.156130783226069, -8.570158796808386 6.158981737634992, -8.599221195241665 6.16220048000181, -8.599858583934925 6.195817658522593, -8.594703363053997 7.056907307318565, -7.774392308954952 7.0175292717254365, -7.83422608263842 6.251657275837222, -7.870735107331853 6.156130783226069))
+712	line_237	LINESTRING (6.9790508758507555 -1.7541124889834099, 7.740071646237887 -1.7705240232309625)
+713	point_237	POINT (-2.8735692698478914 4.2683959344017035)
+714	poly_238	POLYGON ((3.3418135589220275 -3.0923536535461267, 2.6340616394201635 -3.0168045106610704, 2.5683031774107565 -2.3008406891429387, 3.325372515294391 -2.116214510000931, 3.3776615459942008 -2.1796383540801143, 3.374472319914689 -2.9357046651388004, 3.3418135589220275 -3.0923536535461267))
+715	line_238	LINESTRING (0.1312992304819021 5.9789661921784205, -0.7667899476631903 5.919139946922998)
+716	point_238	POINT (6.526519124860317 -2.7036531607785284)
+717	poly_239	POLYGON ((9.216662973315058 3.4093179854296665, 9.083313117100962 3.4098046609352908, 9.066815415163106 3.423198947515745, 9.057680702119482 3.4343781948981436, 9.04493644713933 4.239768712645175, 9.174886997288063 4.281149800002427, 9.19462960109606 4.176695403054401, 9.216662973315058 3.4093179854296665))
+718	line_239	LINESTRING (2.760431611341218 6.787124020927848, 3.6021823087771363 6.80044148265863)
+719	point_239	POINT (8.372820048729235 0.5192540107023008)
+720	poly_240	POLYGON ((0.6520264897627877 9.355777793847281, -0.2588986173628677 9.357823325251175, -0.2812427789644113 9.420202644285126, -0.1241670027331547 10.263472293085593, 0.5115994271783779 10.308363258384441, 0.5777049811057317 10.154685796119592, 0.6384348563704821 9.50666166004926, 0.6520264897627877 9.355777793847281))
+721	line_240	LINESTRING (-2.7464064368890666 -5.060601259407784, -3.564489893227667 -4.900112160001573)
+722	point_240	POINT (9.524970695872312 6.522963509912671)
+723	poly_241	POLYGON ((4.098864629088258 -4.62196299677888, 4.105939321435008 -3.939796838531428, 4.109581316653419 -3.8039565997395, 4.192275563237889 -3.7873549003041798, 5.061998475280259 -3.7719477789012887, 4.998963020706283 -3.8989065073909717, 4.098864629088258 -4.62196299677888))
+724	line_241	LINESTRING (-6.992409415801435 3.7564398742270035, -6.935332241950868 4.599596357580786)
+725	point_241	POINT (-8.987447912005686 9.406360017182276)
+726	poly_242	POLYGON ((1.9085298443228402 8.351800904457184, 1.7938815022708383 8.385380341154933, 1.8137169229752585 9.135777374670013, 1.9418188511378511 9.279811103880501, 2.600959796410659 9.247684729454772, 2.727206663617089 9.15906673246257, 2.772874358685389 8.464325237062294, 2.726297934694999 8.381330150679739, 1.9085298443228402 8.351800904457184))
+727	line_242	LINESTRING (4.867133501196356 -8.030771053862427, 5.6323888572225895 -7.114198405424508)
+728	point_242	POINT (-3.524299199216481 -1.5132048139331729)
+729	poly_243	POLYGON ((9.868038220605357 3.0547325699743553, 9.058172234403397 3.89319579845705, 9.086700608981857 3.9120234020598086, 9.923509429901175 3.9380280391645224, 9.94958180750252 3.9291165826912917, 10.004824778054537 3.8222859512598855, 9.868038220605357 3.0547325699743553))
+730	line_243	LINESTRING (4.560289733506499 -0.9258542216459213, 5.373415489086865 -1.0490351544642207)
+731	point_243	POINT (-8.145316323968002 -2.6240288647464274)
+732	poly_244	POLYGON ((-1.9374224448211346 -6.10793652948027, -2.7182311075821093 -6.102528352431494, -2.822336353765734 -5.220446211477956, -1.9257176972890773 -5.247800957348902, -1.9374224448211346 -6.10793652948027))
+733	line_244	LINESTRING (7.031586053470839 8.809614874423222, 7.956818028607062 8.059258585906662)
+734	point_244	POINT (-3.735389306722654 8.134470951023536)
+735	poly_245	POLYGON ((-7.39525606755642 8.360197110141169, -7.528233522737176 8.387785206371872, -7.43117698531449 9.287526728891534, -6.659782025193677 9.242031775949382, -6.611459165909141 8.403246333156451, -6.619088574910393 8.369991598423582, -7.39525606755642 8.360197110141169))
+736	line_245	LINESTRING (6.051010851545275 7.697237434701982, 6.180767957082829 7.798900926219905)
+737	point_245	POINT (4.227918169851903 1.2319973811511138)
+738	poly_246	POLYGON ((-1.758045690876631 0.4977286632464807, -1.730163976755258 1.155415569537343, -1.0097584470938754 1.3057799035840814, -0.9328558618217824 1.1494955504388529, -1.7224186183812134 0.5028343825643755, -1.758045690876631 0.4977286632464807))
+739	line_246	LINESTRING (6.720862856192552 -1.0182158013470828, 7.511606748395747 -0.1155578599331481)
+740	point_246	POINT (-8.03449942840351 -6.0145953871564455)
+741	poly_247	POLYGON ((7.465212954151603 -3.3068283633543203, 6.690432471389826 -3.273047202543215, 7.319393166536503 -2.3835810416266456, 7.392488764971253 -2.343928359780533, 7.500505697911815 -2.3566581191603877, 7.485811690643425 -3.244833735309187, 7.465212954151603 -3.3068283633543203))
+742	line_247	LINESTRING (-4.65560175472722 4.919357486695315, -4.690377399780605 4.902369382502003)
+743	point_247	POINT (4.740269632351284 2.97757843790648)
+744	poly_248	POLYGON ((10.329641006677997 1.8596249485639746, 9.611546260272693 1.926219542289894, 9.554586091484326 1.9718667667725942, 9.517501402339308 2.6828041386602735, 9.576631908623709 2.7202412927653867, 10.415107051131459 2.6065262633269826, 10.329641006677997 1.8596249485639746))
+745	line_248	LINESTRING (2.39170616882219 7.616600203713868, 1.5944021942779631 6.845662234224759)
+746	point_248	POINT (7.623457696595126 1.2420611103958168)
+747	poly_249	POLYGON ((6.643749653288955 7.66499377623204, 5.8731814000917755 7.69267853067805, 5.877927877814669 8.48084423782797, 5.9652835757853095 8.595847992711176, 6.628154037605432 8.592998404419255, 6.70616123393598 7.730506313163018, 6.643749653288955 7.66499377623204))
+748	line_249	LINESTRING (2.1291477938927565 4.310620458707459, 2.1525699299432426 5.021874295604707)
+749	point_249	POINT (-0.3587010155096166 -2.439207116165887)
+750	poly_250	POLYGON ((7.448084931285977 0.4642822555009884, 6.677396142092296 0.5199301283918852, 6.642995303169561 0.5920553195347928, 6.599251940748144 1.2764346023440913, 6.606607615366846 1.285606700847047, 7.456454988048636 1.431988618963108, 7.448084931285977 0.4642822555009884))
+751	line_250	LINESTRING (9.639161031150184 -9.186972434221284, 9.58406806139348 -9.30006113376925)
+752	point_250	POINT (-0.0041288276231207 4.201127577863675)
+753	poly_251	POLYGON ((3.0021062757781722 6.548999337302843, 2.305557769320897 6.560392922960539, 2.1711711760190973 7.5024904666498164, 3.07030035845434 7.408150223110049, 3.0021062757781722 6.548999337302843))
+754	line_251	LINESTRING (-2.254268367670543 4.485852080167822, -2.301684525580193 3.7782319791892105)
+755	point_251	POINT (-3.8474104372504216 10.06009669189972)
+756	poly_252	POLYGON ((-6.525435838696094 4.89771419082709, -6.577150778245903 4.935741732682226, -7.489670523696037 5.780634499074691, -7.397142798567126 5.820860327288537, -6.641044155004517 5.760838797786383, -6.569441706686099 5.690127469001326, -6.525435838696094 4.89771419082709))
+757	line_252	LINESTRING (-1.2222012117421832 8.812989414991423, -1.2445060224117341 9.61503822569976)
+758	point_252	POINT (7.32779520355179 8.857297752014412)
+759	poly_253	POLYGON ((2.4132745493976606 4.112452885876926, 2.399198746624617 4.136592724183814, 2.351884034582862 4.257356264230362, 2.35315374772112 4.980150272084333, 3.148344861800157 5.056776277899038, 3.1838815404046477 4.9530121731401895, 3.171652220914802 4.228649691297465, 2.4132745493976606 4.112452885876926))
+760	line_253	LINESTRING (5.527121401869584 -4.841347331185239, 5.595386213327146 -4.91504542115428)
+761	point_253	POINT (-0.509780297101201 -4.2564795625876926)
+762	poly_254	POLYGON ((-6.419484457024563 4.782730430665903, -6.562241141451485 5.646656546556563, -5.62472057831726 5.649933348367942, -5.60376338693207 5.496692335422151, -5.775394697707463 4.843685522771845, -6.419484457024563 4.782730430665903))
+763	line_254	LINESTRING (3.8987726465322945 -7.196315517542828, 3.9383279091715444 -7.250183040125213)
+764	point_254	POINT (0.2869858021643121 -3.539507195614596)
+765	poly_255	POLYGON ((10.40499835712801 8.182573213012585, 9.496425061654174 8.188494996371336, 10.306533532245254 9.140228279780102, 10.359878900628868 9.158353571436411, 10.40499835712801 8.182573213012585))
+766	line_255	LINESTRING (3.83975007898353 7.696506683758092, 3.9197991377405432 8.406645404221877)
+767	point_255	POINT (4.964551052910186 1.9622512393357132)
+768	poly_256	POLYGON ((-6.939588741766521 -3.2283846440641115, -6.963906031666079 -2.4081383067891498, -6.139260484602295 -2.36638473922349, -6.201433845751297 -3.148682266758782, -6.939588741766521 -3.2283846440641115))
+769	line_256	LINESTRING (-1.5393098447759184 -1.966870839780327, -1.5282111017732454 -2.6650788610691483)
+770	point_256	POINT (0.1715432094230522 -7.959630819088591)
+771	poly_257	POLYGON ((5.892261360286891 -5.255412181602912, 5.866678268611719 -5.158072353945797, 5.925184345997927 -4.295785146836027, 6.792314380076175 -5.151898293592099, 6.709132406346565 -5.243771885170751, 5.892261360286891 -5.255412181602912))
+772	line_257	LINESTRING (-6.702930068700018 3.649217572846917, -6.6275984897947975 3.7620664846549197)
+773	point_257	POINT (-3.3505182525326944 -0.151062774312656)
+774	poly_258	POLYGON ((-8.817281623086958 -5.113687008822255, -8.981351171833413 -5.096819897303737, -8.941571973330278 -4.34232125178352, -8.848135014614456 -4.295631983994966, -8.794644306162063 -4.300414309608505, -8.009147082193197 -4.380867565866533, -8.030211447698008 -5.082802425554715, -8.817281623086958 -5.113687008822255))
+775	line_258	LINESTRING (-4.6936714883131625 -5.727798147010407, -4.688863991328999 -5.910171297282173)
+776	point_258	POINT (8.466710710049844 -6.184260483777156)
+777	poly_259	POLYGON ((5.131794183748834 -0.0500452832697959, 5.0353765519567855 0.7918161054619951, 5.885026863169759 0.9309353307200876, 5.999156087631849 -0.0134443797677065, 5.131794183748834 -0.0500452832697959))
+778	line_259	LINESTRING (-1.4024580415948826 5.168057810848994, -2.282293768371087 5.259416358704374)
+779	point_259	POINT (4.703035952075408 2.452810846517576)
+780	poly_260	POLYGON ((5.881326832536918 -2.419745337269326, 5.0720847717726585 -2.339683641860355, 4.9137507061864065 -1.6052066702489929, 4.988060300470286 -1.5052693693227905, 5.0589282762542025 -1.4314943845769348, 5.855535956901917 -1.4470237644043733, 5.881326832536918 -2.419745337269326))
+781	line_260	LINESTRING (0.1717825506810905 3.1059078890915557, -0.6767769674170552 3.870593481212159)
+782	point_260	POINT (-4.49751724264109 -1.5940448331581705)
+783	poly_261	POLYGON ((3.4849678588420328 -3.3247911874184193, 2.5983467942058485 -3.3195204423408096, 2.5535417911224734 -3.2363519314907454, 2.61907235195178 -2.5420112195586437, 3.5332364820980975 -2.3658572689438424, 3.4849678588420328 -3.3247911874184193))
+784	line_261	LINESTRING (5.372763857589848 -0.1447727147313609, 4.66259500971181 0.779135972279481)
+785	point_261	POINT (-1.1885281679029605 2.9567915603737394)
+786	poly_262	POLYGON ((0.3124280271610867 5.334207567549091, 0.403242198732842 6.273279480807203, 1.1838855208467627 6.297908058588711, 1.2489086086052699 6.280626778600333, 1.1473790149753285 5.347510220952796, 0.3124280271610867 5.334207567549091))
+787	line_262	LINESTRING (1.6129837555615207 -1.2870865512173815, 2.410310657238872 -2.1423592166666006)
+788	point_262	POINT (-1.5683263379437933 -5.23760061227075)
+789	poly_263	POLYGON ((-5.774655409480059 5.790559813596086, -6.515238866840192 5.810196424703565, -6.524747358223383 6.706339618284084, -5.755073345821016 5.96838458544889, -5.774655409480059 5.790559813596086))
+790	line_263	LINESTRING (-5.525024658706374 4.230000799582859, -5.380196059825321 4.952125178101759)
+791	point_263	POINT (-1.561276171201356 -8.967281381121776)
+792	poly_264	POLYGON ((7.087103365979725 -5.480667778922087, 6.9188420228216225 -4.721284839985449, 6.969423430192266 -4.5724494655667876, 7.705357737693346 -4.582246695641132, 7.900353360983822 -4.596163310199199, 7.087103365979725 -5.480667778922087))
+793	line_264	LINESTRING (-6.751376989947347 -1.300375681846632, -6.942013972208034 -1.2701134952331268)
+794	point_264	POINT (-8.666414200365418 -2.8306691887319477)
+795	poly_265	POLYGON ((-5.655487815815167 7.530140488550671, -6.392435122619851 7.625985380130063, -5.567039148137554 8.480317641742701, -5.543865635424201 7.617223412395542, -5.624463297208611 7.534008213750584, -5.655487815815167 7.530140488550671))
+796	line_265	LINESTRING (-4.129088834410392 9.70104402140729, -4.124154436559785 9.712730525451596)
+797	point_265	POINT (9.7209776346898 10.309207847258737)
+798	poly_266	POLYGON ((0.4691721648142668 -7.611934059630956, -0.368493350957787 -7.509262787870568, -0.4019121615038292 -6.664962190670097, 0.3849020732786885 -6.639976657939181, 0.4768568495086762 -6.657380713093551, 0.5652123145945405 -6.7784597282604295, 0.4691721648142668 -7.611934059630956))
+799	line_266	LINESTRING (1.7631189370279448 3.108955050384354, 0.9001544756135212 2.1340320769917653)
+800	point_266	POINT (2.1600062369485737 -9.476618392932533)
+801	poly_267	POLYGON ((-4.907970643800713 7.175987766690245, -5.736769260315619 7.283973895312272, -4.873785382911873 8.11626626809605, -4.85290497870425 8.031679353279529, -4.815244164923855 7.2783554637932255, -4.907970643800713 7.175987766690245))
+802	line_267	LINESTRING (5.1242850213576965 4.319878332195542, 5.032392398909971 4.3917751729777645)
+803	point_267	POINT (1.7613380466707358 -6.247718199648598)
+804	poly_268	POLYGON ((6.890092518805092 -8.647327642934865, 6.075662413614461 -7.841122154042294, 6.891666500316833 -7.6682327275559805, 6.959870780527249 -7.706739979143319, 6.968225096858045 -7.72476397399888, 7.047530457528044 -8.588172968710944, 6.890092518805092 -8.647327642934865))
+805	line_268	LINESTRING (-2.2096156624654597 -3.3130504261319573, -2.293993220998228 -2.524639670406443)
+806	point_268	POINT (-3.1059209332840365 9.514551726172195)
+807	poly_269	POLYGON ((2.350878260609988 4.7781647920326975, 2.287461348485407 4.824641313219464, 2.3372031627754906 5.666512723930691, 3.0451005739209864 5.572448861673485, 3.070556285294022 5.515900131186305, 3.0512550570306525 4.8364502859066985, 2.350878260609988 4.7781647920326975))
+808	line_269	LINESTRING (-5.325466434724477 9.770494916311693, -4.463239217271608 9.801068708620903)
+809	point_269	POINT (0.5185531135845316 -0.631068258362064)
+810	poly_270	POLYGON ((1.069511595145013 -1.7582054471641333, 1.2197462118548323 -0.8934584160803449, 2.043837121881377 -0.8322943459697278, 2.004366142713762 -1.5886780944080365, 1.948883918048207 -1.6468704978332676, 1.2618064051250855 -1.7357509069893786, 1.069511595145013 -1.7582054471641333))
+811	line_270	LINESTRING (5.036021719093616 -1.909750101546875, 4.938036101040812 -1.7400880537578964)
+812	point_270	POINT (9.532520593027238 -6.906348654370372)
+813	poly_271	POLYGON ((-9.537220084250952 -3.475743144139503, -10.206554969910464 -3.406112992157606, -10.311789446510058 -2.6162131884616358, -9.502305398723308 -2.6089176588843985, -9.402509973673116 -2.6854584252511273, -9.537220084250952 -3.475743144139503))
+814	line_271	LINESTRING (0.2978680154309092 4.3810879216108205, -0.3423649186957812 5.214598392812379)
+815	point_271	POINT (-9.785542830738187 -9.731383589516437)
+816	poly_272	POLYGON ((5.237320520173289 1.4628167312069817, 5.232402409692618 2.336996486934978, 5.296949254891381 2.3920680340112024, 6.093148645683054 2.319129465857016, 6.041515789255967 1.5063467008195541, 5.237320520173289 1.4628167312069817))
+817	line_272	LINESTRING (-3.405245976617132 -5.434850812827753, -2.672662835213462 -4.5953170688592335)
+818	point_272	POINT (-2.634105070361503 3.287709725334957)
+819	poly_273	POLYGON ((-7.433034928624383 -8.28809217698734, -8.249367043283849 -8.187370327288916, -8.356806106677336 -7.35729431009033, -7.400288234928456 -7.373585086367371, -7.433034928624383 -8.28809217698734))
+820	line_273	LINESTRING (-10.31689363557306 -1.6959825172535667, -10.309592774148356 -1.772496001200905)
+821	point_273	POINT (-5.549347860595968 1.7095561114730804)
+822	poly_274	POLYGON ((-9.739010255085562 -4.596964023938786, -9.857981550644697 -3.7417768356775096, -9.791004156812305 -3.681258143892423, -9.032748207317102 -3.8364919883360136, -8.969222996740154 -4.5824989879653435, -9.739010255085562 -4.596964023938786))
+823	line_274	LINESTRING (-5.500765537959382 6.76233740357374, -5.603502725083772 6.626228900418018)
+824	point_274	POINT (8.03453638342627 2.672184929848384)
+825	poly_275	POLYGON ((-7.905862177236617 -9.358543984476027, -7.907315245555594 -8.406151772080644, -7.760446270068248 -8.41207678260694, -6.967668565884415 -8.459166079903083, -7.879441794257489 -9.333284115431027, -7.905862177236617 -9.358543984476027))
+826	line_275	LINESTRING (0.9794647714778336 -8.200555737218314, 1.0682888420889602 -8.111133353152068)
+827	point_275	POINT (4.751292339864884 -0.7624092502905692)
+828	poly_276	POLYGON ((2.682032222492949 2.6736489603086895, 2.6790516576363017 3.548774620650393, 2.720986754956706 3.6165840422511004, 3.5047411351864737 3.6403575205790486, 3.5055666101056455 3.4781391751938426, 2.682032222492949 2.6736489603086895))
+829	line_276	LINESTRING (7.8953165337689075 3.654055433524645, 7.071655062795754 3.6941954471833736)
+830	point_276	POINT (2.358949372763868 -6.8661281768217926)
+831	poly_277	POLYGON ((6.65011280628365 -8.778465199118129, 6.484815192358856 -8.674063412462754, 6.510307921140325 -7.892390157751124, 7.453648459080906 -7.897142301450205, 6.65011280628365 -8.778465199118129))
+832	line_277	LINESTRING (4.956666022897755 2.484144235296158, 5.762700321133015 1.7207509428226282)
+833	point_277	POINT (2.6606321445051995 3.2627607905405007)
+834	poly_278	POLYGON ((-0.0957617579547738 -9.5756272653984, -0.0714836827658614 -8.653520848262373, 0.7981989207790837 -8.628462668120614, 0.812269345535325 -9.465713252398718, -0.0957617579547738 -9.5756272653984))
+835	line_278	LINESTRING (-7.1702648218650635 -2.241056947687752, -7.05542362334233 -2.2305246084424115)
+836	point_278	POINT (-6.782167765240859 3.2547948767509283)
+837	poly_279	POLYGON ((0.4302500113829001 8.343124895412535, 0.3995993689615844 9.213605654157504, 1.1672755037920162 9.300640426008314, 1.3112308654262919 8.528874244195798, 1.2561827364556717 8.40247289292048, 0.4302500113829001 8.343124895412535))
+838	line_279	LINESTRING (2.344874790947251 5.289902016626796, 3.157884133099392 5.3057950275277275)
+839	point_279	POINT (0.0025737129246025 -7.805746098464787)
+840	poly_280	POLYGON ((-3.079371537267248 1.5726296896616947, -3.1060790374505935 1.5856867799049175, -3.1513230759933384 1.7221765397633118, -3.084445062068184 2.4060354057564584, -2.3327489811311013 2.508696672362356, -2.262345070789475 1.6562686314739454, -3.079371537267248 1.5726296896616947))
+841	line_280	LINESTRING (6.032828931282676 -0.0774805225055979, 6.136790108549498 -0.0877680716972718)
+842	point_280	POINT (-0.4642585987608661 -9.731433847716378)
+843	poly_281	POLYGON ((1.0125231598033464 6.4866853945948195, 0.2240701433138171 6.65223965225327, 0.0902296098701201 7.4077093278436985, 0.9213658085401164 7.471195108825854, 0.9885660479060392 7.3792773949743, 1.0125231598033464 6.4866853945948195))
+844	line_281	LINESTRING (2.879303020615285 -3.2188864314654113, 2.7777983166435956 -3.936257783037143)
+845	point_281	POINT (8.45244454805542 9.063497552128887)
+846	poly_282	POLYGON ((2.0074081506537738 7.0118600474293835, 1.965492100383787 7.201832038361833, 2.682932521035851 7.950516194066224, 2.8313932135158812 7.90663705395526, 2.82055252956187 7.080024788187015, 2.7070436835781573 7.069425373516357, 2.0074081506537738 7.0118600474293835))
+847	line_282	LINESTRING (7.838101909911593 -1.8186935286566217, 7.070499910495155 -1.110283070888302)
+848	point_282	POINT (4.583031911632924 7.295902030836667)
+849	poly_283	POLYGON ((-9.13149692468584 6.7500608783758125, -9.134328494108676 6.752321711787408, -9.16332734281522 6.781694817198216, -9.191614260278428 7.57615893868901, -9.126831755787979 7.6274932231059225, -8.337772852690733 7.584631629089242, -9.13149692468584 6.7500608783758125))
+850	line_283	LINESTRING (9.576511443582245 -3.018713440553049, 9.663073500273999 -2.9417355154939773)
+851	point_283	POINT (7.99874348063839 -0.8449703000258919)
+852	poly_284	POLYGON ((-2.26463905684895 -1.5451304841029827, -3.125084798271929 -1.5404222057702137, -3.106033697353388 -0.6540334825959458, -2.2989892221120245 -0.5859323368853393, -2.262857354924652 -1.4112747269497272, -2.26463905684895 -1.5451304841029827))
+853	line_284	LINESTRING (1.180038959631648 -5.791833529526898, 0.5469933272904208 -6.478394763522478)
+854	point_284	POINT (-6.216033745923838 -6.408400925298)
+855	poly_285	POLYGON ((-0.6314897712209386 -0.8521373912603398, -1.4318834719575677 -0.8059796286664505, -1.555680149409905 -0.7533372460465608, -0.769340918734056 0.0686285927793088, -0.6382276222709196 0.0961135646642539, -0.6314897712209386 -0.8521373912603398))
+856	line_285	LINESTRING (-9.134702712582355 4.421253984585118, -8.236760968740912 3.520404475823108)
+857	point_285	POINT (3.599134918419238 0.570944438944049)
+858	poly_286	POLYGON ((2.6492725661744063 7.94601919264288, 1.6612864737475568 7.972174926723678, 1.7075871001337855 8.728494420012218, 1.8036305559554968 8.824934911245858, 2.6062576593485405 8.907413973626738, 2.6492725661744063 7.94601919264288))
+859	line_286	LINESTRING (0.1401968705910442 -4.3673128471624825, 1.0399883151468254 -4.390790723831738)
+860	point_286	POINT (8.621806912890639 2.1946960088233856)
+861	poly_287	POLYGON ((-3.4329271299249777 4.741906465270895, -4.285731837013271 4.802471887247939, -4.3094671588421285 5.567197958040314, -4.25133928963092 5.6773385986534635, -3.4788630347494554 5.662415122921325, -3.4017288373660968 5.64905908508193, -3.4329271299249777 4.741906465270895))
+862	line_287	LINESTRING (3.3520055164321496 -1.6444757205626184, 4.132153830898011 -2.5141953183183534)
+863	point_287	POINT (2.2870896359327553 9.150944441288807)
+864	poly_288	POLYGON ((-7.433048587144241 -9.325627696838396, -7.582084050921665 -9.253427078300357, -7.553412760598912 -8.421631964168, -7.521585059455727 -8.344865195185978, -6.601964391425233 -8.467251339752085, -7.433048587144241 -9.325627696838396))
+865	line_288	LINESTRING (7.349040080000717 -8.391275035780097, 6.514490439570643 -8.38650338942167)
+866	point_288	POINT (8.445212133432669 8.166615515988314)
+867	poly_289	POLYGON ((4.1135353046603225 9.279425109522538, 3.289530099002664 10.00887294937231, 3.284831135567675 10.022082419938132, 3.3871838827111063 10.072953085651893, 4.172072545543349 10.173219638054253, 4.250249195391294 10.115941770656764, 4.1135353046603225 9.279425109522538))
+868	line_289	LINESTRING (3.07362185413069 3.0349628719122284, 3.0511204890678423 3.055289073754268)
+869	point_289	POINT (0.790626014849166 -4.860766549063067)
+870	poly_290	POLYGON ((1.8703771672143623 -3.864374626208337, 1.8898171026529211 -2.9741334980643663, 2.72343634392481 -3.020689506422297, 1.9757919683053047 -3.8406159362734256, 1.8703771672143623 -3.864374626208337))
+871	line_290	LINESTRING (3.0573669341436034 6.623171516648422, 3.0693841059046076 7.528862660647217)
+872	point_290	POINT (-3.4301791503612464 -7.710591816972441)
+873	poly_291	POLYGON ((-3.3848336435861546 -10.097702840887976, -4.2461904839449405 -10.054185905466547, -4.279974393936474 -9.4010327799758, -4.166755305604793 -9.317067069330207, -3.44268906254474 -9.253470997630176, -3.3835436302295694 -9.272570605342167, -3.3848336435861546 -10.097702840887976))
+874	line_291	LINESTRING (8.612880760695338 -2.0007558878929546, 8.67947756442882 -1.3830306831120158)
+875	point_291	POINT (3.230680590778268 3.787166418248261)
+876	poly_292	POLYGON ((-9.219509069221468 -0.776078354214744, -9.982243086796878 -0.7727623218701714, -10.108219219806212 -0.5919590003541195, -10.077448361674593 0.0765460989016107, -10.046661897856506 0.1082648004591253, -9.172933682278956 0.169019035915742, -9.173791629218808 0.103477576821831, -9.219509069221468 -0.776078354214744))
+877	line_292	LINESTRING (-10.169711319607872 9.040568565922504, -9.4573811201992 9.16198363262577)
+878	point_292	POINT (-0.6158567815159721 2.187541807638195)
+879	poly_293	POLYGON ((-0.6814691609278931 -9.632831812277404, -0.8160850222925411 -9.612788832642275, -1.630224142344601 -8.748057513745856, -1.6052649217242658 -8.729055854198819, -0.7225859503481123 -8.791190036320929, -0.6814691609278931 -9.632831812277404))
+880	line_293	LINESTRING (-4.60230662328407 1.1960432746841296, -5.385415343344709 1.2611416809868166)
+881	point_293	POINT (-0.6995291876293486 -4.112834080729116)
+882	poly_294	POLYGON ((-0.9704699341964044 3.550501496565834, -1.8303649072779697 3.5563652790984492, -1.8871304612025552 3.576197431102833, -1.9648268583075108 4.404057352304367, -1.041121395673389 4.4443583270398195, -0.9704699341964044 3.550501496565834))
+883	line_294	LINESTRING (-6.629615966923989 -5.602814110291847, -6.75871733798409 -4.787004964759064)
+884	point_294	POINT (-7.389554018603032 -7.646807703463196)
+885	poly_295	POLYGON ((3.2484337166452306 0.9264358696947281, 3.2222876938531346 1.7260086952377913, 3.2689953153893376 1.784140946396718, 4.070447983210581 1.7277973392826038, 4.086934819374747 1.6811503044906488, 4.148006104464677 0.9815010643589773, 3.9540405512845 0.9634024654349224, 3.2484337166452306 0.9264358696947281))
+886	line_295	LINESTRING (-1.069503755833642 -6.479878634675283, -1.7994781139174378 -6.441410126730174)
+887	point_295	POINT (1.9570964106200588 -0.0314879302333931)
+888	poly_296	POLYGON ((8.678889989370719 7.279949786397513, 8.669778550650493 7.340799278467358, 8.680320152733628 7.410063249260491, 9.552652116254096 8.253464645378033, 9.654189033008981 7.28455119145555, 8.678889989370719 7.279949786397513))
+889	line_296	LINESTRING (-2.8808074370736416 -2.083637012472012, -2.771082284407311 -2.0855068433157813)
+890	point_296	POINT (-9.777119299799555 10.161690295145927)
+891	poly_297	POLYGON ((5.035694303050028 -9.935315223641721, 4.200803431394263 -9.776382415915947, 4.941856621794712 -8.98130226152104, 5.060181006388283 -8.977777807515855, 5.090432583952324 -8.988979601117407, 5.035694303050028 -9.935315223641721))
+892	line_297	LINESTRING (2.0239433956488666 6.480419199870548, 2.0444478557296994 5.720999180578932)
+893	point_297	POINT (2.749431803736594 -3.195953944003208)
+894	poly_298	POLYGON ((9.588314984712342 -0.8144444231370944, 9.479562956611268 -0.720803758471046, 8.715715350335547 0.057186498379084, 8.676233856107306 0.1307118025306638, 9.634106486327141 0.0734915382973015, 9.588314984712342 -0.8144444231370944))
+895	line_298	LINESTRING (-10.116113992738734 0.5130645498048496, -9.96958539589582 0.5427630137900196)
+896	point_298	POINT (1.1872987752927364 -6.781716235552693)
+897	poly_299	POLYGON ((2.50863685606555 -1.4978666195212795, 1.8011372483104042 -1.3913909965966296, 2.5282677176064485 -0.5541860641601986, 2.59997080585911 -0.5894793546604994, 2.6896997335144057 -1.4103851415713857, 2.682001270073766 -1.4476201326062839, 2.50863685606555 -1.4978666195212795))
+898	line_299	LINESTRING (-7.377093006607241 -2.8148087406915865, -7.333487662036741 -3.6921560146082575)
+899	point_299	POINT (-7.044598875820627 6.288818779434376)
+900	poly_m0	POLYGON ((10 10, 11 10, 11 11, 10 11, 10 10))
+901	poly_m1	POLYGON ((0 10, 1 10, 1 11, 0 11, 0 10))
+902	line_m2	LINESTRING (10 10, 11 11)
+903	point_m3	POINT (10 10)
+904	point_m4	POINT (-10 -10)
+905	poly_m5	POLYGON ((9 9, 11 9, 11 11, 9 11, 9 9))
+906	line_m6	LINESTRING (9 9, 11 11)
diff --git a/core/src/test/scala/org/apache/sedona/core/scalaTest.scala b/core/src/test/scala/org/apache/sedona/core/scalaTest.scala
index 94ce205f..18206adc 100644
--- a/core/src/test/scala/org/apache/sedona/core/scalaTest.scala
+++ b/core/src/test/scala/org/apache/sedona/core/scalaTest.scala
@@ -22,6 +22,7 @@ package org.apache.sedona.core
 import org.apache.sedona.core.enums.{FileDataSplitter, GridType, IndexType, JoinBuildSide}
 import org.apache.sedona.core.formatMapper.EarthdataHDFPointMapper
 import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.spatialOperator.{JoinQuery, KNNQuery, RangeQuery}
 import org.apache.sedona.core.spatialRDD.{CircleRDD, PointRDD, PolygonRDD}
 import org.apache.spark.storage.StorageLevel
@@ -57,6 +58,13 @@ class scalaTest extends SparkUtil {
     objectRDDcopy.analyze()
   }
 
+  test("should pass spatial range query with spatial predicate") {
+    val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
+    for (i <- 1 to eachQueryLoopTimes) {
+      val resultSize = RangeQuery.SpatialRangeQuery(objectRDD, rangeQueryWindow, SpatialPredicate.COVERED_BY, false).count
+    }
+  }
+
   test("should pass spatial range query") {
     val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
     for (i <- 1 to eachQueryLoopTimes) {
@@ -87,6 +95,18 @@ class scalaTest extends SparkUtil {
     }
   }
 
+  test("should pass spatial join query with spatial predicate") {
+    val queryWindowRDD = new PolygonRDD(sc, PolygonRDDInputLocation, PolygonRDDStartOffset, PolygonRDDEndOffset, PolygonRDDSplitter, true)
+    val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
+    objectRDD.analyze()
+    objectRDD.spatialPartitioning(joinQueryPartitioningType)
+    queryWindowRDD.spatialPartitioning(objectRDD.getPartitioner)
+
+    for (i <- 1 to eachQueryLoopTimes) {
+      val resultSize = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, false, SpatialPredicate.INTERSECTS).count
+    }
+  }
+
   test("should pass spatial join query") {
     val queryWindowRDD = new PolygonRDD(sc, PolygonRDDInputLocation, PolygonRDDStartOffset, PolygonRDDEndOffset, PolygonRDDSplitter, true)
     val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
@@ -152,6 +172,18 @@ class scalaTest extends SparkUtil {
     }
   }
 
+  test("should pass distance join query with spatial predicate") {
+    val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
+    val queryWindowRDD = new CircleRDD(objectRDD, 0.1)
+    objectRDD.analyze()
+    objectRDD.spatialPartitioning(GridType.QUADTREE)
+    queryWindowRDD.spatialPartitioning(objectRDD.getPartitioner)
+
+    for (i <- 1 to eachQueryLoopTimes) {
+      val resultSize = JoinQuery.DistanceJoinQuery(objectRDD, queryWindowRDD, false, SpatialPredicate.INTERSECTS).count()
+    }
+  }
+
   test("should pass distance join query") {
     val objectRDD = new PointRDD(sc, PointRDDInputLocation, PointRDDOffset, PointRDDSplitter, false)
     val queryWindowRDD = new CircleRDD(objectRDD, 0.1)
diff --git a/docs/tutorial/rdd.md b/docs/tutorial/rdd.md
index f45b3fe0..8ea06ee6 100644
--- a/docs/tutorial/rdd.md
+++ b/docs/tutorial/rdd.md
@@ -237,26 +237,36 @@ val rddWithOtherAttributes = objectRDD.rawSpatialRDD.rdd.map[String](f=>f.getUse
 
 ## Write a Spatial Range Query
 
-A spatial range query takes as input a range query window and an SpatialRDD and returns all geometries that intersect / are fully covered by the query window.
+A spatial range query takes as input a range query window and an SpatialRDD and returns all geometries that have specified relationship with the query window.
 
 
 Assume you now have an SpatialRDD (typed or generic). You can use the following code to issue an Spatial Range Query on it.
 
 ```Scala
 val rangeQueryWindow = new Envelope(-90.01, -80.01, 30.01, 40.01)
-val considerBoundaryIntersection = false // Only return gemeotries fully covered by the window
+val spatialPredicate = SpatialPredicate.COVERED_BY // Only return gemeotries fully covered by the window
 val usingIndex = false
-var queryResult = RangeQuery.SpatialRangeQuery(spatialRDD, rangeQueryWindow, considerBoundaryIntersection, usingIndex)
+var queryResult = RangeQuery.SpatialRangeQuery(spatialRDD, rangeQueryWindow, spatialPredicate, usingIndex)
 ```
 
-==considerBoundaryIntersection== can be set to TRUE to return all geometries intersect with query window.
+==spatialPredicate== can be set to `SpatialPredicate.INTERSECTS` to return all geometries intersect with query window. Supported spatial predicates are:
+
+* `CONTAINS`: geometry is completely inside the query window
+* `INTERSECTS`: geometry have at least one point in common with the query window
+* `WITHIN`: geometry is completely within the query window (no touching edges)
+* `COVERS`: query window has no point outside of the geometry
+* `COVERED_BY`: geometry has no point outside of the query window
+* `OVERLAPS`: geometry and the query window spatially overlap
+* `CROSSES`: geometry and the query window spatially cross
+* `TOUCHES`: the only points shared between geometry and the query window are on the boundary of geometry and the query window
+* `EQUALS`: geometry and the query window are spatially equal
 
 !!!note
-	Spatial range query is equal to ==ST_Within== and ==ST_Intersects== in Spatial SQL. An example query is as follows:
+	Spatial range query is equivalent with a SELECT query with spatial predicate as search condition in Spatial SQL. An example query is as follows:
 	```SQL
 	SELECT *
 	FROM checkin
-	WHERE ST_Intersects(queryWindow, checkin.location)
+	WHERE ST_Intersects(checkin.location, queryWindow)
 	```
 
 ### Range query window
@@ -303,13 +313,13 @@ To utilize a spatial index in a spatial range query, use the following code:
 
 ```Scala
 val rangeQueryWindow = new Envelope(-90.01, -80.01, 30.01, 40.01)
-val considerBoundaryIntersection = false // Only return gemeotries fully covered by the window
+val spatialPredicate = SpatialPredicate.COVERED_BY // Only return gemeotries fully covered by the window
 
 val buildOnSpatialPartitionedRDD = false // Set to TRUE only if run join query
 spatialRDD.buildIndex(IndexType.QUADTREE, buildOnSpatialPartitionedRDD)
 
 val usingIndex = true
-var queryResult = RangeQuery.SpatialRangeQuery(spatialRDD, rangeQueryWindow, considerBoundaryIntersection, usingIndex)
+var queryResult = RangeQuery.SpatialRangeQuery(spatialRDD, rangeQueryWindow, spatialPredicate, usingIndex)
 ```
 
 !!!tip
@@ -382,7 +392,7 @@ A spatial join query takes as input two Spatial RDD A and B. For each geometry i
 Assume you now have two SpatialRDDs (typed or generic). You can use the following code to issue an Spatial Join Query on them.
 
 ```Scala
-val considerBoundaryIntersection = false // Only return gemeotries fully covered by each query window in queryWindowRDD
+val spatialPredicate = SpatialPredicate.COVERED_BY // Only return gemeotries fully covered by each query window in queryWindowRDD
 val usingIndex = false
 
 objectRDD.analyze()
@@ -390,7 +400,7 @@ objectRDD.analyze()
 objectRDD.spatialPartitioning(GridType.KDBTREE)
 queryWindowRDD.spatialPartitioning(objectRDD.getPartitioner)
 
-val result = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, usingIndex, considerBoundaryIntersection)
+val result = JoinQuery.SpatialJoinQuery(objectRDD, queryWindowRDD, usingIndex, spatialPredicate)
 ```
 
 !!!note
@@ -433,7 +443,7 @@ val buildOnSpatialPartitionedRDD = true // Set to TRUE only if run join query
 val usingIndex = true
 queryWindowRDD.buildIndex(IndexType.QUADTREE, buildOnSpatialPartitionedRDD)
 
-val result = JoinQuery.SpatialJoinQueryFlat(objectRDD, queryWindowRDD, usingIndex, considerBoundaryIntersection)
+val result = JoinQuery.SpatialJoinQueryFlat(objectRDD, queryWindowRDD, usingIndex, spatialPredicate)
 ```
 
 The index should be built on either one of two SpatialRDDs. In general, you should build it on the larger SpatialRDD.
@@ -468,13 +478,13 @@ val circleRDD = new CircleRDD(objectRddA, 0.1) // Create a CircleRDD using the g
 circleRDD.spatialPartitioning(GridType.KDBTREE)
 objectRddB.spatialPartitioning(circleRDD.getPartitioner)
 
-val considerBoundaryIntersection = false // Only return gemeotries fully covered by each query window in queryWindowRDD
+val spatialPredicate = SpatialPredicate.COVERED_BY // Only return gemeotries fully covered by each query window in queryWindowRDD
 val usingIndex = false
 
-val result = JoinQuery.DistanceJoinQueryFlat(objectRddB, circleRDD, usingIndex, considerBoundaryIntersection)
+val result = JoinQuery.DistanceJoinQueryFlat(objectRddB, circleRDD, usingIndex, spatialPredicate)
 ```
 
-The rest part of the join query is same as the spatial join query.
+Distance join can only accept `COVERED_BY` and `INTERSECTS` as spatial predicates. The rest part of the join query is same as the spatial join query.
 
 The details of spatial partitioning in join query is [here](#use-spatial-partitioning).
 
diff --git a/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java b/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java
index 2a6bacb3..c5a31be2 100644
--- a/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java
+++ b/flink/src/main/java/org/apache/sedona/flink/expressions/Predicates.java
@@ -15,7 +15,8 @@ package org.apache.sedona.flink.expressions;
 
 import org.apache.flink.table.annotation.DataTypeHint;
 import org.apache.flink.table.functions.ScalarFunction;
-import org.apache.sedona.core.joinJudgement.JudgementHelper;
+import org.apache.sedona.core.joinJudgement.JoinConditionMatcher;
+import org.apache.sedona.core.spatialOperator.SpatialPredicate;
 import org.apache.sedona.core.spatialPartitioning.PartitioningUtils;
 import org.apache.sedona.core.utils.HalfOpenRectangle;
 import org.locationtech.jts.geom.Envelope;
@@ -27,6 +28,7 @@ import java.util.Objects;
 public class Predicates {
     public static class ST_Intersects extends ScalarFunction {
         private List<Envelope> grids;
+        private static final JoinConditionMatcher MATCHER = JoinConditionMatcher.create(SpatialPredicate.INTERSECTS);
 
         /**
          * Constructor for duplicate removal
@@ -61,12 +63,13 @@ public class Predicates {
             Geometry geom1 = (Geometry) o1;
             Geometry geom2 = (Geometry) o2;
             HalfOpenRectangle halfOpenRectangle = new HalfOpenRectangle(grids.get(key));
-            return JudgementHelper.match(geom1, geom2, halfOpenRectangle, true);
+            return MATCHER.match(geom1, geom2, halfOpenRectangle);
         }
     }
 
     public static class ST_Contains extends ScalarFunction {
         private List<Envelope> grids;
+        private static final JoinConditionMatcher MATCHER = JoinConditionMatcher.create(SpatialPredicate.CONTAINS);
 
         /**
          * Constructor for duplicate removal
@@ -101,12 +104,13 @@ public class Predicates {
             Geometry geom1 = (Geometry) o1;
             Geometry geom2 = (Geometry) o2;
             HalfOpenRectangle halfOpenRectangle = new HalfOpenRectangle(grids.get(key));
-            return JudgementHelper.match(geom1, geom2, halfOpenRectangle, false);
+            return MATCHER.match(geom1, geom2, halfOpenRectangle);
         }
     }
 
     public static class ST_Within extends ScalarFunction {
         private List<Envelope> grids;
+        private static final JoinConditionMatcher MATCHER = JoinConditionMatcher.create(SpatialPredicate.WITHIN);
 
         /**
          * Constructor for duplicate removal
@@ -141,12 +145,13 @@ public class Predicates {
             Geometry geom1 = (Geometry) o1;
             Geometry geom2 = (Geometry) o2;
             HalfOpenRectangle halfOpenRectangle = new HalfOpenRectangle(grids.get(key));
-            return JudgementHelper.match(geom1, geom2, halfOpenRectangle, true);
+            return MATCHER.match(geom1, geom2, halfOpenRectangle);
         }
     }
 
     public static class ST_Covers extends ScalarFunction {
         private List<Envelope> grids;
+        private static final JoinConditionMatcher MATCHER = JoinConditionMatcher.create(SpatialPredicate.COVERS);
 
         /**
          * Constructor for duplicate removal
@@ -181,12 +186,13 @@ public class Predicates {
             Geometry geom1 = (Geometry) o1;
             Geometry geom2 = (Geometry) o2;
             HalfOpenRectangle halfOpenRectangle = new HalfOpenRectangle(grids.get(key));
-            return JudgementHelper.match(geom1, geom2, halfOpenRectangle, false);
+            return MATCHER.match(geom1, geom2, halfOpenRectangle);
         }
     }
 
     public static class ST_CoveredBy extends ScalarFunction {
         private List<Envelope> grids;
+        private static final JoinConditionMatcher MATCHER = JoinConditionMatcher.create(SpatialPredicate.COVERED_BY);
 
         /**
          * Constructor for duplicate removal
@@ -221,7 +227,7 @@ public class Predicates {
             Geometry geom1 = (Geometry) o1;
             Geometry geom2 = (Geometry) o2;
             HalfOpenRectangle halfOpenRectangle = new HalfOpenRectangle(grids.get(key));
-            return JudgementHelper.match(geom1, geom2, halfOpenRectangle, true);
+            return MATCHER.match(geom1, geom2, halfOpenRectangle);
         }
     }
 
diff --git a/python/tests/sql/test_predicate_join.py b/python/tests/sql/test_predicate_join.py
index 8aee3506..ed049113 100644
--- a/python/tests/sql/test_predicate_join.py
+++ b/python/tests/sql/test_predicate_join.py
@@ -110,7 +110,7 @@ class TestPredicateJoin(TestBase):
 
         range_join_df.explain()
         range_join_df.show(3)
-        assert range_join_df.count() == 1000
+        assert range_join_df.count() == 0
 
     def test_st_within_in_a_join(self):
         polygon_csv_df = self.spark.read.format("csv").option("delimiter", ",").option("header", "false").load(
@@ -165,7 +165,7 @@ class TestPredicateJoin(TestBase):
 
         range_join_df.explain()
         range_join_df.show(3)
-        assert range_join_df.count() == 57
+        assert range_join_df.count() == 15
 
     def test_st_crosses_in_a_join(self):
         polygon_csv_df = self.spark.read.format("csv").\
@@ -199,7 +199,7 @@ class TestPredicateJoin(TestBase):
 
         range_join_df.explain()
         range_join_df.show(3)
-        assert range_join_df.count() == 1000
+        assert range_join_df.count() == 0
 
     def test_st_distance_radius_in_a_join(self):
         point_csv_df_1 = self.spark.read.format("csv").\
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastIndexJoinExec.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastIndexJoinExec.scala
index 345ea76e..f60b8947 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastIndexJoinExec.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/BroadcastIndexJoinExec.scala
@@ -18,8 +18,10 @@
  */
 package org.apache.spark.sql.sedona_sql.strategy.join
 
-import collection.JavaConverters._
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
+import org.apache.sedona.core.spatialOperator.SpatialPredicateEvaluators
 
+import collection.JavaConverters._
 import org.apache.sedona.core.spatialRDD.SpatialRDD
 import org.apache.spark.broadcast.Broadcast
 import org.apache.spark.internal.Logging
@@ -38,7 +40,7 @@ case class BroadcastIndexJoinExec(left: SparkPlan,
                                   streamShape: Expression,
                                   indexBuildSide: JoinSide,
                                   windowJoinSide: JoinSide,
-                                  intersects: Boolean,
+                                  spatialPredicate: SpatialPredicate,
                                   extraCondition: Option[Expression] = None,
                                   radius: Option[Expression] = None)
   extends SedonaBinaryExecNode
@@ -69,12 +71,11 @@ case class BroadcastIndexJoinExec(left: SparkPlan,
     (streamShape, broadcast.shape)
   }
 
-  private val spatialExpression = radius match {
-      case Some(r) if intersects => s"ST_Distance($windowExpression, $objectExpression) <= $r"
-      case Some(r) if !intersects => s"ST_Distance($windowExpression, $objectExpression) < $r"
-      case None if intersects => s"ST_Intersects($windowExpression, $objectExpression)"
-      case None if !intersects => s"ST_Contains($windowExpression, $objectExpression)"
-    }
+  private val spatialExpression = (radius, spatialPredicate) match {
+    case (Some(r), SpatialPredicate.INTERSECTS) => s"ST_Distance($windowExpression, $objectExpression) <= $r"
+    case (Some(r), _) => s"ST_Distance($windowExpression, $objectExpression) < $r"
+    case (None, _) => s"ST_$spatialPredicate($windowExpression, $objectExpression)"
+  }
 
   override def simpleString(maxFields: Int): String = super.simpleString(maxFields) + s" $spatialExpression" // SPARK3 anchor
 //  override def simpleString: String = super.simpleString + s" $spatialExpression" // SPARK2 anchor
@@ -82,8 +83,9 @@ case class BroadcastIndexJoinExec(left: SparkPlan,
   private def windowBroadcastJoin(index: Broadcast[SpatialIndex], spatialRdd: SpatialRDD[Geometry]): RDD[(Geometry, Geometry)] = {
     spatialRdd.getRawSpatialRDD.rdd.flatMap { row =>
       val candidates = index.value.query(row.getEnvelopeInternal).iterator.asScala.asInstanceOf[Iterator[Geometry]]
+      val evaluator = SpatialPredicateEvaluators.create(spatialPredicate)
       candidates
-        .filter(candidate => if (intersects) candidate.intersects(row) else candidate.covers(row))
+        .filter(candidate => evaluator.eval(candidate, row))
         .map(candidate => (candidate, row))
     }
   }
@@ -91,8 +93,9 @@ case class BroadcastIndexJoinExec(left: SparkPlan,
   private def objectBroadcastJoin(index: Broadcast[SpatialIndex], spatialRdd: SpatialRDD[Geometry]): RDD[(Geometry, Geometry)] = {
     spatialRdd.getRawSpatialRDD.rdd.flatMap { row =>
       val candidates = index.value.query(row.getEnvelopeInternal).iterator.asScala.asInstanceOf[Iterator[Geometry]]
+      val evaluator = SpatialPredicateEvaluators.create(spatialPredicate)
       candidates
-        .filter(candidate => if (intersects) row.intersects(candidate) else row.covers(candidate))
+        .filter(candidate => evaluator.eval(row, candidate))
         .map(candidate => (row, candidate))
     }
   }
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/DistanceJoinExec.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/DistanceJoinExec.scala
index 645fbfcc..ebaf0005 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/DistanceJoinExec.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/DistanceJoinExec.scala
@@ -19,6 +19,7 @@
 package org.apache.spark.sql.sedona_sql.strategy.join
 
 import org.apache.sedona.common.geometryObjects.Circle
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.spatialRDD.SpatialRDD
 import org.apache.sedona.sql.utils.GeometrySerializer
 import org.apache.spark.internal.Logging
@@ -36,7 +37,7 @@ case class DistanceJoinExec(left: SparkPlan,
                             leftShape: Expression,
                             rightShape: Expression,
                             radius: Expression,
-                            intersects: Boolean,
+                            spatialPredicate: SpatialPredicate,
                             extraCondition: Option[Expression] = None)
   extends SedonaBinaryExecNode
     with TraitJoinQueryExec
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
index 79b24264..6a020e33 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala
@@ -19,6 +19,7 @@
 package org.apache.spark.sql.sedona_sql.strategy.join
 
 import org.apache.sedona.core.enums.IndexType
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.utils.SedonaConf
 import org.apache.spark.sql.{SparkSession, Strategy}
 import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan, LessThanOrEqual}
@@ -28,13 +29,12 @@ import org.apache.spark.sql.execution.SparkPlan
 import org.apache.spark.sql.sedona_sql.expressions._
 
 
-
 case class JoinQueryDetection(
   left: LogicalPlan,
   right: LogicalPlan,
   leftShape: Expression,
   rightShape: Expression,
-  intersects: Boolean,
+  spatialPredicate: SpatialPredicate,
   extraCondition: Option[Expression] = None,
   radius: Option[Expression] = None
 )
@@ -45,7 +45,7 @@ case class JoinQueryDetection(
   *
   * Plans `DistanceJoinExec` for inner joins on spatial relationship ST_Distance(a, b) < r.
   * 
-  * Plans `BroadcastIndexJoinExec for inner joins on spatial relationships with a broadcast hint.
+  * Plans `BroadcastIndexJoinExec` for inner joins on spatial relationships with a broadcast hint.
   */
 class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
 
@@ -56,23 +56,23 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
     extraCondition: Option[Expression] = None): Option[JoinQueryDetection] = {
       predicate match {
         case ST_Contains(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.CONTAINS, extraCondition))
         case ST_Intersects(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, true, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.INTERSECTS, extraCondition))
         case ST_Within(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(right, left, rightShape, leftShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.WITHIN, extraCondition))
         case ST_Covers(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.COVERS, extraCondition))
         case ST_CoveredBy(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(right, left, rightShape, leftShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.COVERED_BY, extraCondition))
         case ST_Overlaps(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(right, left, rightShape, leftShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.OVERLAPS, extraCondition))
         case ST_Touches(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, true, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.TOUCHES, extraCondition))
         case ST_Equals(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.EQUALS, extraCondition))
         case ST_Crosses(Seq(leftShape, rightShape)) =>
-          Some(JoinQueryDetection(right, left, rightShape, leftShape, false, extraCondition))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.CROSSES, extraCondition))
         case _ => None
       }
     }
@@ -93,17 +93,17 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
         case Some(And(extraCondition, predicate: ST_Predicate)) =>
           getJoinDetection(left, right, predicate, Some(extraCondition))
         case Some(LessThanOrEqual(ST_Distance(Seq(leftShape, rightShape)), radius)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, true, None, Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.INTERSECTS, None, Some(radius)))
         case Some(And(LessThanOrEqual(ST_Distance(Seq(leftShape, rightShape)), radius), extraCondition)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, true, Some(extraCondition), Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.INTERSECTS, Some(extraCondition), Some(radius)))
         case Some(And(extraCondition, LessThanOrEqual(ST_Distance(Seq(leftShape, rightShape)), radius))) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, true, Some(extraCondition), Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.INTERSECTS, Some(extraCondition), Some(radius)))
         case Some(LessThan(ST_Distance(Seq(leftShape, rightShape)), radius)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, None, Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.COVERS, None, Some(radius)))
         case Some(And(LessThan(ST_Distance(Seq(leftShape, rightShape)), radius), extraCondition)) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, Some(extraCondition), Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.COVERS, Some(extraCondition), Some(radius)))
         case Some(And(extraCondition, LessThan(ST_Distance(Seq(leftShape, rightShape)), radius))) =>
-          Some(JoinQueryDetection(left, right, leftShape, rightShape, false, Some(extraCondition), Some(radius)))
+          Some(JoinQueryDetection(left, right, leftShape, rightShape, SpatialPredicate.COVERS, Some(extraCondition), Some(radius)))
         case _ =>
           None
       }
@@ -112,17 +112,17 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
 
       if ((broadcastLeft || broadcastRight) && sedonaConf.getUseIndex) {
         queryDetection match {
-          case Some(JoinQueryDetection(left, right, leftShape, rightShape, intersects, extraCondition, radius)) =>
-            planBroadcastJoin(left, right, Seq(leftShape, rightShape), intersects, sedonaConf.getIndexType, broadcastLeft, extraCondition, radius)
+          case Some(JoinQueryDetection(left, right, leftShape, rightShape, spatialPredicate, extraCondition, radius)) =>
+            planBroadcastJoin(left, right, Seq(leftShape, rightShape), spatialPredicate, sedonaConf.getIndexType, broadcastLeft, extraCondition, radius)
           case _ =>
             Nil
         }
       } else {
         queryDetection match {
-          case Some(JoinQueryDetection(left, right, leftShape, rightShape, intersects, extraCondition, None)) =>
-            planSpatialJoin(left, right, Seq(leftShape, rightShape), intersects, extraCondition)
-          case Some(JoinQueryDetection(left, right, leftShape, rightShape, intersects, extraCondition, Some(radius))) =>
-            planDistanceJoin(left, right, Seq(leftShape, rightShape), radius, intersects, extraCondition)
+          case Some(JoinQueryDetection(left, right, leftShape, rightShape, spatialPredicate, extraCondition, None)) =>
+            planSpatialJoin(left, right, Seq(leftShape, rightShape), spatialPredicate, extraCondition)
+          case Some(JoinQueryDetection(left, right, leftShape, rightShape, spatialPredicate, extraCondition, Some(radius))) =>
+            planDistanceJoin(left, right, Seq(leftShape, rightShape), radius, spatialPredicate, extraCondition)
           case None => 
             Nil
         }
@@ -137,8 +137,7 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
     * map to the output of the specified plan.
     */
   private def matches(expr: Expression, plan: LogicalPlan): Boolean =
-    expr.references.find(plan.outputSet.contains(_)).isDefined &&
-      expr.references.find(!plan.outputSet.contains(_)).isEmpty
+    expr.references.nonEmpty && expr.references.forall(plan.outputSet.contains(_))
 
   private def matchExpressionsToPlans(exprA: Expression,
                                       exprB: Expression,
@@ -155,17 +154,17 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
   private def planSpatialJoin(left: LogicalPlan,
                               right: LogicalPlan,
                               children: Seq[Expression],
-                              intersects: Boolean,
+                              spatialPredicate: SpatialPredicate,
                               extraCondition: Option[Expression] = None): Seq[SparkPlan] = {
     val a = children.head
     val b = children.tail.head
 
-    val relationship = if (intersects) "ST_Intersects" else "ST_Contains";
+    val relationship = s"ST_$spatialPredicate"
 
     matchExpressionsToPlans(a, b, left, right) match {
       case Some((planA, planB, _)) =>
         logInfo(s"Planning spatial join for $relationship relationship")
-        RangeJoinExec(planLater(planA), planLater(planB), a, b, intersects, extraCondition) :: Nil
+        RangeJoinExec(planLater(planA), planLater(planB), a, b, spatialPredicate, extraCondition) :: Nil
       case None =>
         logInfo(
           s"Spatial join for $relationship with arguments not aligned " +
@@ -178,21 +177,19 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
                                right: LogicalPlan,
                                children: Seq[Expression],
                                radius: Expression,
-                               intersects: Boolean,
+                               spatialPredicate: SpatialPredicate,
                                extraCondition: Option[Expression] = None): Seq[SparkPlan] = {
     val a = children.head
     val b = children.tail.head
 
-    val relationship = if (intersects) "ST_Distance <=" else "ST_Distance <";
-
     matchExpressionsToPlans(a, b, left, right) match {
       case Some((planA, planB, _)) =>
         if (radius.references.isEmpty || matches(radius, planA)) {
           logInfo("Planning spatial distance join")
-          DistanceJoinExec(planLater(planA), planLater(planB), a, b, radius, intersects, extraCondition) :: Nil
+          DistanceJoinExec(planLater(planA), planLater(planB), a, b, radius, spatialPredicate, extraCondition) :: Nil
         } else if (matches(radius, planB)) {
           logInfo("Planning spatial distance join")
-          DistanceJoinExec(planLater(planB), planLater(planA), b, a, radius, intersects, extraCondition) :: Nil
+          DistanceJoinExec(planLater(planB), planLater(planA), b, a, radius, spatialPredicate, extraCondition) :: Nil
         } else {
           logInfo(
             "Spatial distance join for ST_Distance with non-scalar radius " +
@@ -210,7 +207,7 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
   private def planBroadcastJoin(left: LogicalPlan,
                                 right: LogicalPlan,
                                 children: Seq[Expression],
-                                intersects: Boolean,
+                                spatialPredicate: SpatialPredicate,
                                 indexType: IndexType,
                                 broadcastLeft: Boolean,
                                 extraCondition: Option[Expression],
@@ -218,11 +215,10 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
     val a = children.head
     val b = children.tail.head
 
-    val relationship = radius match {
-      case Some(_) if intersects => "ST_Distance <="
-      case Some(_) if !intersects => "ST_Distance <"
-      case None if intersects => "ST_Intersects"
-      case None if !intersects => "ST_Contains"
+    val relationship = (radius, spatialPredicate) match {
+      case (Some(_), SpatialPredicate.INTERSECTS) => "ST_Distance <="
+      case (Some(_), _) => "ST_Distance <"
+      case (None, _) => s"ST_$spatialPredicate"
     }
 
     matchExpressionsToPlans(a, b, left, right) match {
@@ -239,7 +235,7 @@ class JoinQueryDetector(sparkSession: SparkSession) extends Strategy {
           case (RightSide, true) => // Broadcast the right side, objects on the left
             (planLater(left), SpatialIndexExec(planLater(right), a, indexType, radius), b, RightSide)
         }
-        BroadcastIndexJoinExec(leftPlan, rightPlan, streamShape, broadcastSide, windowSide, intersects, extraCondition, radius) :: Nil
+        BroadcastIndexJoinExec(leftPlan, rightPlan, streamShape, broadcastSide, windowSide, spatialPredicate, extraCondition, radius) :: Nil
       case None =>
         logInfo(
           s"Spatial join for $relationship with arguments not aligned " +
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/RangeJoinExec.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/RangeJoinExec.scala
index 2fe11fa5..feffe99d 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/RangeJoinExec.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/RangeJoinExec.scala
@@ -18,6 +18,7 @@
  */
 package org.apache.spark.sql.sedona_sql.strategy.join
 
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.execution.SparkPlan
@@ -39,7 +40,7 @@ case class RangeJoinExec(left: SparkPlan,
                          right: SparkPlan,
                          leftShape: Expression,
                          rightShape: Expression,
-                         intersects: Boolean,
+                         spatialPredicate: SpatialPredicate,
                          extraCondition: Option[Expression] = None)
   extends SedonaBinaryExecNode
     with TraitJoinQueryExec
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryExec.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryExec.scala
index d0f59ae9..ca5418b3 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryExec.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryExec.scala
@@ -21,6 +21,7 @@ package org.apache.spark.sql.sedona_sql.strategy.join
 import org.apache.sedona.core.enums.JoinSparitionDominantSide
 import org.apache.sedona.core.spatialOperator.JoinQuery
 import org.apache.sedona.core.spatialOperator.JoinQuery.JoinParams
+import org.apache.sedona.core.spatialOperator.SpatialPredicate
 import org.apache.sedona.core.utils.SedonaConf
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
@@ -45,7 +46,7 @@ trait TraitJoinQueryExec extends TraitJoinQueryBase {
   val right: SparkPlan
   val leftShape: Expression
   val rightShape: Expression
-  val intersects: Boolean
+  val spatialPredicate: SpatialPredicate
   val extraCondition: Option[Expression]
 
   override def output: Seq[Attribute] = left.output ++ right.output
@@ -119,7 +120,7 @@ trait TraitJoinQueryExec extends TraitJoinQueryBase {
     }
 
 
-    val joinParams = new JoinParams(sedonaConf.getUseIndex, intersects, sedonaConf.getIndexType, sedonaConf.getJoinBuildSide)
+    val joinParams = new JoinParams(sedonaConf.getUseIndex, spatialPredicate, sedonaConf.getIndexType, sedonaConf.getJoinBuildSide)
 
     //logInfo(s"leftShape count ${leftShapes.spatialPartitionedRDD.count()}")
     //logInfo(s"rightShape count ${rightShapes.spatialPartitionedRDD.count()}")
diff --git a/sql/src/test/scala/org/apache/sedona/sql/SpatialJoinSuite.scala b/sql/src/test/scala/org/apache/sedona/sql/SpatialJoinSuite.scala
new file mode 100644
index 00000000..42da9063
--- /dev/null
+++ b/sql/src/test/scala/org/apache/sedona/sql/SpatialJoinSuite.scala
@@ -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.sedona.sql
+
+import org.apache.sedona.common.geometryObjects.Circle
+import org.apache.spark.sql.Column
+import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.functions.col
+import org.apache.spark.sql.sedona_sql.expressions.st_constructors.ST_GeomFromText
+import org.apache.spark.sql.types.IntegerType
+import org.locationtech.jts.geom.Geometry
+import org.locationtech.jts.io.WKTReader
+import org.scalatest.prop.TableDrivenPropertyChecks
+
+class SpatialJoinSuite extends TestBaseScala with TableDrivenPropertyChecks {
+
+  val testDataDelimiter = "\t"
+  val spatialJoinPartitionSideConfKey = "sedona.join.spatitionside"
+
+  describe("Sedona-SQL Spatial Join Test") {
+    val joinConditions = Table("join condition",
+      "ST_Contains(df1.geom, df2.geom)",
+      "ST_Intersects(df1.geom, df2.geom)",
+      "ST_Within(df1.geom, df2.geom)",
+      "ST_Covers(df1.geom, df2.geom)",
+      "ST_CoveredBy(df1.geom, df2.geom)",
+      "ST_Touches(df1.geom, df2.geom)",
+      "ST_Crosses(df1.geom, df2.geom)",
+      "ST_Overlaps(df1.geom, df2.geom)",
+      "ST_Equals(df1.geom, df2.geom)",
+
+      "ST_Contains(df2.geom, df1.geom)",
+      "ST_Intersects(df2.geom, df1.geom)",
+      "ST_Within(df2.geom, df1.geom)",
+      "ST_Covers(df2.geom, df1.geom)",
+      "ST_CoveredBy(df2.geom, df1.geom)",
+      "ST_Touches(df2.geom, df1.geom)",
+      "ST_Crosses(df2.geom, df1.geom)",
+      "ST_Overlaps(df2.geom, df1.geom)",
+      "ST_Equals(df2.geom, df1.geom)",
+
+      "ST_Distance(df1.geom, df2.geom) < 1.0",
+      "ST_Distance(df1.geom, df2.geom) <= 1.0",
+      "ST_Distance(df2.geom, df1.geom) < 1.0",
+      "ST_Distance(df2.geom, df1.geom) <= 1.0"
+    )
+
+    var spatialJoinPartitionSide = "left"
+    try {
+      spatialJoinPartitionSide = sparkSession.sparkContext.getConf.get(spatialJoinPartitionSideConfKey, "left")
+      forAll (joinConditions) { joinCondition =>
+        it(s"should join two dataframes with $joinCondition") {
+          sparkSession.sparkContext.getConf.set(spatialJoinPartitionSideConfKey, "left")
+          prepareTempViewsForTestData()
+          val result = sparkSession.sql(s"SELECT df1.id, df2.id FROM df1 JOIN df2 ON $joinCondition")
+          val expected = buildExpectedResult(joinCondition)
+          verifyResult(expected, result)
+        }
+        it(s"should join two dataframes with $joinCondition, with right side as dominant side") {
+          sparkSession.sparkContext.getConf.set(spatialJoinPartitionSideConfKey, "right")
+          prepareTempViewsForTestData()
+          val result = sparkSession.sql(s"SELECT df1.id, df2.id FROM df1 JOIN df2 ON $joinCondition")
+          val expected = buildExpectedResult(joinCondition)
+          verifyResult(expected, result)
+        }
+        it(s"should join two dataframes with $joinCondition, broadcast the left side") {
+          prepareTempViewsForTestData()
+          val result = sparkSession.sql(s"SELECT /*+ BROADCAST(df1) */ df1.id, df2.id FROM df1 JOIN df2 ON $joinCondition")
+          val expected = buildExpectedResult(joinCondition)
+          verifyResult(expected, result)
+        }
+        it(s"should join two dataframes with $joinCondition, broadcast the right side") {
+          prepareTempViewsForTestData()
+          val result = sparkSession.sql(s"SELECT /*+ BROADCAST(df2) */ df1.id, df2.id FROM df1 JOIN df2 ON $joinCondition")
+          val expected = buildExpectedResult(joinCondition)
+          verifyResult(expected, result)
+        }
+      }
+    } finally {
+      sparkSession.sparkContext.getConf.set(spatialJoinPartitionSideConfKey, spatialJoinPartitionSide)
+    }
+  }
+
+  private def prepareTempViewsForTestData(): (DataFrame, DataFrame) = {
+    val df1 = sparkSession.read.format("csv").option("header", "false").option("delimiter", testDataDelimiter)
+      .load(spatialJoinLeftInputLocation)
+      .withColumn("id", col("_c0").cast(IntegerType))
+      .withColumn("geom", ST_GeomFromText(new Column("_c2")))
+    val df2 = sparkSession.read.format("csv").option("header", "false").option("delimiter", testDataDelimiter)
+      .load(spatialJoinRightInputLocation)
+      .withColumn("id", col("_c0").cast(IntegerType))
+      .withColumn("geom", ST_GeomFromText(new Column("_c2")))
+    df1.createOrReplaceTempView("df1")
+    df2.createOrReplaceTempView("df2")
+    (df1, df2)
+  }
+
+  private def buildExpectedResult(joinCondition: String): Seq[(Int, Int)] = {
+    val left = loadTestData(spatialJoinLeftInputLocation)
+    val right = loadTestData(spatialJoinRightInputLocation)
+    val udf = joinCondition.split('(')(0)
+    val swapped = joinCondition.contains("df2.geom, df1.geom")
+    val eval = udf match {
+      case "ST_Contains" => (l: Geometry, r: Geometry) => l.contains(r)
+      case "ST_CoveredBy" => (l: Geometry, r: Geometry) => l.coveredBy(r)
+      case "ST_Covers" => (l: Geometry, r: Geometry) => l.covers(r)
+      case "ST_Crosses" => (l: Geometry, r: Geometry) => l.crosses(r)
+      case "ST_Equals" => (l: Geometry, r: Geometry) => l.equals(r)
+      case "ST_Intersects" => (l: Geometry, r: Geometry) => l.intersects(r)
+      case "ST_Overlaps" => (l: Geometry, r: Geometry) => l.overlaps(r)
+      case "ST_Touches" => (l: Geometry, r: Geometry) => l.touches(r)
+      case "ST_Within" => (l: Geometry, r: Geometry) => l.within(r)
+      case "ST_Distance" =>
+        // XXX: ST_Distance has a weird behavior, it is wildly different from `l.distance(r)`.
+        if (joinCondition.contains("<=")) {
+          (l: Geometry, r: Geometry) => new Circle(l, 1.0).intersects(r)
+        } else {
+          (l: Geometry, r: Geometry) => new Circle(l, 1.0).covers(r)
+        }
+    }
+    left.flatMap { case (id, geom) =>
+      right.filter { case (_, geom2) =>
+        if (swapped) eval(geom2, geom) else eval(geom, geom2)
+      }.map { case (id2, _) => (id, id2) }
+    }.sorted
+  }
+
+  private def loadTestData(path: String): Seq[(Int, Geometry)] = {
+    val wktReader = new WKTReader()
+    val bufferedSource = scala.io.Source.fromFile(path)
+    try {
+      bufferedSource.getLines().map { line =>
+        val Array(id, _, geom) = line.split(testDataDelimiter)
+        (id.toInt, wktReader.read(geom))
+      }.toList
+    } finally {
+      bufferedSource.close()
+    }
+  }
+
+  def verifyResult(expected: Seq[(Int, Int)], result: DataFrame): Unit = {
+    val actual = result.collect().map(row => (row.getInt(0), row.getInt(1))).sorted
+    assert(actual.nonEmpty)
+    assert(actual === expected)
+  }
+}
diff --git a/sql/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala b/sql/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
index 5cf0c718..be3ae5f8 100644
--- a/sql/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
+++ b/sql/src/test/scala/org/apache/sedona/sql/TestBaseScala.scala
@@ -61,6 +61,8 @@ trait TestBaseScala extends FunSpec with BeforeAndAfterAll {
   val geojsonIdInputLocation = resourceFolder + "testContainsId.json"
   val smallAreasLocation: String = resourceFolder + "small/areas.csv"
   val smallPointsLocation: String = resourceFolder + "small/points.csv"
+  val spatialJoinLeftInputLocation: String = resourceFolder + "spatial-predicates-test-data.tsv"
+  val spatialJoinRightInputLocation: String = resourceFolder + "spatial-join-query-window.tsv"
 
   override def beforeAll(): Unit = {
     SedonaSQLRegistrator.registerAll(sparkSession)
diff --git a/sql/src/test/scala/org/apache/sedona/sql/predicateJoinTestScala.scala b/sql/src/test/scala/org/apache/sedona/sql/predicateJoinTestScala.scala
index 221724a6..45ec24aa 100644
--- a/sql/src/test/scala/org/apache/sedona/sql/predicateJoinTestScala.scala
+++ b/sql/src/test/scala/org/apache/sedona/sql/predicateJoinTestScala.scala
@@ -76,7 +76,7 @@ class predicateJoinTestScala extends TestBaseScala {
 
       var rangeJoinDf = sparkSession.sql("select * from polygondf, pointdf where ST_Touches(polygondf.polygonshape,pointdf.pointshape) ")
 
-      assert(rangeJoinDf.count() == 1000)
+      assert(rangeJoinDf.count() == 0)
     }
 
     it("Passed ST_Within in a join") {
@@ -108,7 +108,7 @@ class predicateJoinTestScala extends TestBaseScala {
 
       var rangeJoinDf = sparkSession.sql("select * from polygondf, polygonodf where ST_Overlaps(polygondf.polygonshape, polygonodf.polygonshape)")
 
-      assert(rangeJoinDf.count() == 57)
+      assert(rangeJoinDf.count() == 15)
     }
 
     it("Passed ST_Crosses in a join") {
@@ -124,7 +124,7 @@ class predicateJoinTestScala extends TestBaseScala {
 
       var rangeJoinDf = sparkSession.sql("select * from polygondf, pointdf where ST_Crosses(pointdf.pointshape, polygondf.polygonshape) ")
 
-      assert(rangeJoinDf.count() == 1000)
+      assert(rangeJoinDf.count() == 0)
     }
 
     it("Passed ST_Covers in a join") {