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") {