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 2020/12/03 08:40:17 UTC
[incubator-sedona] branch master updated: [SEDONA-1] Move to jts
1.18, minimize the dependency on JTSplus (#488)
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 4800676 [SEDONA-1] Move to jts 1.18, minimize the dependency on JTSplus (#488)
4800676 is described below
commit 4800676929d3d07db2f7e6421a643ffd562f90a3
Author: Jia Yu <ji...@apache.org>
AuthorDate: Thu Dec 3 01:40:06 2020 -0700
[SEDONA-1] Move to jts 1.18, minimize the dependency on JTSplus (#488)
* Fix some test cases according to JTS updates
* Fix some test cases according to JTS updates
* Update which submodule commit to track
* Remove the SNAPSHOT tag in JTS dependency
* Add the findLeafFunctions
* Update which submodule commit to track
* Clean up the code
* Remove the dependency on Geometry "Equals", remove all partitioning methods except Quad-Tree and KDB-Tree, remove all HashSet dependency
Signed-off-by: Jia Yu <ji...@gmail.com>
* Fix the ScalaExample in Viz
* Fix the scala test
* Refactor IndexSerde
* Update the jts dependency to the latest
* Upgrade to JTS 1.18.0-SNAPSHOT and GeoTools 24.0
* Fix the JTS2GeoJSON issue on JTS 1.17+
* Fix the bug in the adapter
* Finalize the package private solution for IndexSerde
Signed-off-by: Jia Yu <ji...@gmail.com>
* Update Sedona PR according to the changes in JTS PR
* Fix the logic in SpatialJoinQuery to make it preserve all duplicates
* Remove additional grid types.
* Bring back geom and wkt for uniqueness checking.
* Remove hash from variables name.
* Speed up the HashCode computation in JoinQuery collect
* Fix the null issue
* Fix the out of memory test failure.
Co-authored-by: Pawel Kocinski <pa...@gmail.com>
---
.gitmodules | 2 +-
core/pom.xml | 4 +-
.../org/apache/sedona/core/enums/GridType.java | 29 +--
.../apache/sedona/core/geometryObjects/Circle.java | 13 +-
.../core/geometryObjects/GeoJSONWriterNew.java | 121 ++++++++++++
.../core/geometryObjects/SpatialIndexSerde.java | 205 ++-------------------
.../sedona/core/spatialOperator/JoinQuery.java | 163 ++++------------
.../spatialPartitioning/RtreePartitioning.java | 49 ++++-
.../apache/sedona/core/spatialRDD/SpatialRDD.java | 46 +----
.../org/apache/sedona/core/utils/GeomUtils.java | 46 +++++
.../org/apache/sedona/core/utils/SedonaConf.java | 2 +-
.../jts/index/quadtree/IndexSerde.java | 115 ++++++++++++
.../locationtech/jts/index/strtree/IndexSerde.java | 155 ++++++++++++++++
.../shapefileParser/shapes/GeometrySerdeTest.java | 10 +-
.../shapefileParser/shapes/ShapefileRDDTest.java | 11 +-
.../shapes/ShapefileReaderTest.java | 9 +-
.../sedona/core/geometryObjects/CircleTest.java | 10 +-
.../JoinQueryCorrectnessChecker.java | 57 +++---
.../sedona/core/spatialOperator/JoinTestBase.java | 31 +---
.../core/spatialOperator/LineStringJoinTest.java | 24 +--
.../sedona/core/spatialOperator/PointJoinTest.java | 28 +--
.../sedona/core/spatialOperator/PointKnnTest.java | 2 -
.../core/spatialOperator/PointRangeTest.java | 2 -
.../core/spatialOperator/PolygonJoinTest.java | 20 +-
.../core/spatialOperator/RectangleJoinTest.java | 24 +--
.../sedona/core/spatialRDD/LineStringRDDTest.java | 50 -----
.../sedona/core/spatialRDD/PointRDDTest.java | 70 -------
.../sedona/core/spatialRDD/PolygonRDDTest.java | 50 -----
.../sedona/core/spatialRDD/RectangleRDDTest.java | 53 ------
.../core/spatialRDD/SpatialRDDWriterTest.java | 19 +-
.../sedona/core/utils/CRSTransformationTest.java | 17 +-
.../test/resources/babylon.linestring.properties | 9 -
core/src/test/resources/babylon.point.properties | 9 -
core/src/test/resources/babylon.polygon.properties | 9 -
.../test/resources/babylon.rectangle.properties | 9 -
core/src/test/resources/crs.test.properties | 2 +-
core/src/test/resources/linestring.test.properties | 2 +-
core/src/test/resources/point.test.properties | 2 +-
core/src/test/resources/polygon.test.properties | 2 +-
core/src/test/resources/rectangle.test.properties | 2 +-
jts | 2 +-
pom.xml | 2 +-
.../adapters/PythonConverter.scala | 6 +-
...dConverter.scala => ListPairRddConverter.scala} | 4 +-
.../utils/implicits.scala | 16 +-
.../sedona/python/wrapper/GeometrySample.scala | 4 +-
.../python/wrapper/TestToPythonSerialization.scala | 21 ++-
python/sedona/core/SpatialRDD/__init__.py | 2 +-
python/sedona/core/SpatialRDD/linestring_rdd.py | 7 +-
python/sedona/core/SpatialRDD/point_rdd.py | 8 +-
python/sedona/core/SpatialRDD/polygon_rdd.py | 19 +-
python/sedona/core/SpatialRDD/rectangle_rdd.py | 19 +-
python/sedona/core/SpatialRDD/spatial_rdd.py | 17 +-
.../sedona/core/SpatialRDD/spatial_rdd_factory.py | 1 -
python/sedona/core/enums/file_data_splitter.py | 2 -
python/sedona/core/enums/grid_type.py | 6 -
python/sedona/core/enums/index_type.py | 2 -
python/sedona/core/enums/join_build_side.py | 2 +-
python/sedona/core/formatMapper/disc_utils.py | 3 +-
python/sedona/core/formatMapper/geo_json_reader.py | 3 +-
python/sedona/core/formatMapper/geo_reader.py | 7 +-
.../shapefileParser/shape_file_reader.py | 12 --
python/sedona/core/formatMapper/wkb_reader.py | 15 +-
python/sedona/core/formatMapper/wkt_reader.py | 8 +-
python/sedona/core/geom/circle.py | 2 +-
python/sedona/core/jvm/abstract.py | 1 -
python/sedona/core/jvm/config.py | 4 +
python/sedona/core/jvm/partitioner.py | 83 ---------
python/sedona/core/jvm/translate.py | 4 +-
python/sedona/core/spatialOperator/join_query.py | 25 +--
python/sedona/core/spatialOperator/knn_query.py | 2 +-
python/sedona/core/spatialOperator/range_query.py | 5 +-
python/sedona/core/utils.py | 1 -
python/sedona/exceptions.py | 2 -
python/sedona/register/java_libs.py | 2 +-
python/sedona/utils/abstract_parser.py | 1 -
python/sedona/utils/binary_parser.py | 7 +-
python/sedona/utils/decorators.py | 8 +-
python/sedona/utils/jvm.py | 2 +-
python/sedona/utils/meta.py | 4 +-
python/sedona/utils/spatial_rdd_parser.py | 6 +-
python/sedona/utils/types.py | 1 -
python/setup.py | 2 +-
python/tests/core/test_core_rdd.py | 2 +-
python/tests/core/test_core_spatial_relations.py | 4 +-
python/tests/core/test_spatial_rdd_from_disc.py | 4 +-
python/tests/properties/crs_transform.py | 3 +-
python/tests/properties/linestring_properties.py | 2 +-
python/tests/properties/polygon_properties.py | 5 +-
python/tests/resources/points.csv | 2 +-
python/tests/spatial_operator/test_join_base.py | 14 +-
.../test_join_query_correctness.py | 2 +-
.../tests/spatial_operator/test_linestring_join.py | 29 +--
python/tests/spatial_operator/test_point_join.py | 80 ++++----
python/tests/spatial_operator/test_point_knn.py | 2 -
python/tests/spatial_operator/test_point_range.py | 6 -
python/tests/spatial_operator/test_polygon_join.py | 32 ++--
.../tests/spatial_operator/test_rectangle_join.py | 29 +--
python/tests/spatial_rdd/test_linestring_rdd.py | 45 -----
python/tests/spatial_rdd/test_point_rdd.py | 59 +-----
python/tests/spatial_rdd/test_polygon_rdd.py | 46 -----
python/tests/spatial_rdd/test_rectangle_rdd.py | 56 ------
python/tests/spatial_rdd/test_spatial_rdd.py | 15 +-
.../tests/spatial_rdd/test_spatial_rdd_writer.py | 2 +-
python/tests/utils/test_crs_transformation.py | 17 +-
.../org/apache/sedona/sql/utils/Adapter.scala | 17 +-
.../sql/sedona_sql/expressions/Functions.scala | 5 +-
.../test/resources/babylon.linestring.properties | 9 -
sql/src/test/resources/babylon.point.properties | 9 -
sql/src/test/resources/babylon.polygon.properties | 9 -
.../test/resources/babylon.rectangle.properties | 9 -
sql/src/test/resources/crs.test.properties | 2 +-
sql/src/test/resources/linestring.test.properties | 10 +-
sql/src/test/resources/point.test.properties | 16 +-
sql/src/test/resources/polygon.test.properties | 16 +-
sql/src/test/resources/rectangle.test.properties | 10 +-
sql/src/test/resources/scalastyle_config.xml | 45 ++---
.../org/apache/sedona/viz/showcase/Example.java | 4 +-
.../apache/sedona/viz/showcase/ScalaExample.scala | 4 +-
.../org/apache/sedona/viz/ChoroplethmapTest.java | 8 +-
.../org/apache/sedona/viz/rdd/scalaTest.scala | 4 +-
121 files changed, 1003 insertions(+), 1463 deletions(-)
diff --git a/.gitmodules b/.gitmodules
index 3701fc1..98c374c 100644
--- a/.gitmodules
+++ b/.gitmodules
@@ -4,4 +4,4 @@
[submodule "jts"]
path = jts
url = https://github.com/jiayuasu/jts.git
- branch = 1.16.x
+ branch = add-getter-setter
diff --git a/core/pom.xml b/core/pom.xml
index e60772a..95210a0 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -37,13 +37,13 @@
<dependency>
<groupId>org.locationtech.jts</groupId>
<artifactId>jts-core</artifactId>
- <version>1.16.2-SNAPSHOT</version>
+ <version>1.18.0-SNAPSHOT</version>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>org.wololo</groupId>
<artifactId>jts2geojson</artifactId>
- <version>0.13.0</version>
+ <version>0.14.3</version>
<exclusions>
<exclusion>
<groupId>com.fasterxml.jackson.core</groupId>
diff --git a/core/src/main/java/org/apache/sedona/core/enums/GridType.java b/core/src/main/java/org/apache/sedona/core/enums/GridType.java
index 9efdced..2a95c08 100755
--- a/core/src/main/java/org/apache/sedona/core/enums/GridType.java
+++ b/core/src/main/java/org/apache/sedona/core/enums/GridType.java
@@ -19,6 +19,9 @@
package org.apache.sedona.core.enums;
+import org.apache.log4j.Logger;
+import org.apache.sedona.core.formatMapper.FormatMapper;
+
import java.io.Serializable;
// TODO: Auto-generated Javadoc
@@ -29,33 +32,13 @@ import java.io.Serializable;
public enum GridType
implements Serializable
{
-
- /**
- * The equalgrid.
- */
- EQUALGRID,
-
- /**
- * The hilbert.
- */
- HILBERT,
-
- /**
- * The rtree.
- */
- RTREE,
-
- /**
- * The voronoi.
- */
- VORONOI,
/**
- * The voronoi.
+ * The Quad-Tree partitioning.
*/
QUADTREE,
/**
- * K-D-B-tree (k-dimensional B-tree)
+ * K-D-B-tree partitioning (k-dimensional B-tree)
*/
KDBTREE;
@@ -67,9 +50,11 @@ public enum GridType
*/
public static GridType getGridType(String str)
{
+ final Logger logger = Logger.getLogger(GridType.class);
for (GridType me : GridType.values()) {
if (me.name().equalsIgnoreCase(str)) { return me; }
}
+ logger.error("[Sedona] Choose quadtree or kdbtree instead. This grid type is not supported: " + str);
return null;
}
}
diff --git a/core/src/main/java/org/apache/sedona/core/geometryObjects/Circle.java b/core/src/main/java/org/apache/sedona/core/geometryObjects/Circle.java
index 961a5d5..6e26c36 100644
--- a/core/src/main/java/org/apache/sedona/core/geometryObjects/Circle.java
+++ b/core/src/main/java/org/apache/sedona/core/geometryObjects/Circle.java
@@ -19,6 +19,7 @@
package org.apache.sedona.core.geometryObjects;
+import org.apache.sedona.core.utils.GeomUtils;
import org.locationtech.jts.geom.Coordinate;
import org.locationtech.jts.geom.CoordinateFilter;
import org.locationtech.jts.geom.CoordinateSequenceComparator;
@@ -372,6 +373,14 @@ public class Circle
return newCircle;
}
+ @Override
+ protected Geometry reverseInternal()
+ {
+ Geometry g = this.centerGeometry.reverse();
+ Circle newCircle = new Circle(g, this.radius);
+ return newCircle;
+ }
+
public Geometry copy()
{
Circle cloneCircle = new Circle(this.centerGeometry.copy(), this.radius);
@@ -398,7 +407,7 @@ public class Circle
if (type1 != type2) { return false; }
if (radius1 != radius2) { return false; }
- return this.centerGeometry.equals(((Circle) g).centerGeometry);
+ return GeomUtils.equalsTopoGeom(this.centerGeometry, ((Circle) g).centerGeometry);
}
/* (non-Javadoc)
@@ -499,7 +508,7 @@ public class Circle
}
@Override
- protected int getSortIndex()
+ protected int getTypeCode()
{
return 0;
}
diff --git a/core/src/main/java/org/apache/sedona/core/geometryObjects/GeoJSONWriterNew.java b/core/src/main/java/org/apache/sedona/core/geometryObjects/GeoJSONWriterNew.java
new file mode 100644
index 0000000..225498a
--- /dev/null
+++ b/core/src/main/java/org/apache/sedona/core/geometryObjects/GeoJSONWriterNew.java
@@ -0,0 +1,121 @@
+/**
+ * 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.geometryObjects;
+
+import org.locationtech.jts.geom.Coordinate;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.GeometryCollection;
+import org.locationtech.jts.geom.LineString;
+import org.locationtech.jts.geom.MultiLineString;
+import org.locationtech.jts.geom.MultiPoint;
+import org.locationtech.jts.geom.MultiPolygon;
+import org.locationtech.jts.geom.Point;
+import org.locationtech.jts.geom.Polygon;
+import org.wololo.jts2geojson.GeoJSONReader;
+
+public class GeoJSONWriterNew
+{
+ final static GeoJSONReader reader = new GeoJSONReader();
+
+ public org.wololo.geojson.Geometry write(Geometry geometry) {
+ Class<? extends Geometry> c = geometry.getClass();
+ if (c.equals(Point.class)) {
+ return convert((Point) geometry);
+ } else if (c.equals(LineString.class)) {
+ return convert((LineString) geometry);
+ } else if (c.equals(Polygon.class)) {
+ return convert((Polygon) geometry);
+ } else if (c.equals(MultiPoint.class)) {
+ return convert((MultiPoint) geometry);
+ } else if (c.equals(MultiLineString.class)) {
+ return convert((MultiLineString) geometry);
+ } else if (c.equals(MultiPolygon.class)) {
+ return convert((MultiPolygon) geometry);
+ } else if (c.equals(GeometryCollection.class)) {
+ return convert((GeometryCollection) geometry);
+ } else {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ org.wololo.geojson.Point convert(Point point) {
+ org.wololo.geojson.Point json = new org.wololo.geojson.Point(
+ convert(point.getCoordinate()));
+ return json;
+ }
+
+ org.wololo.geojson.MultiPoint convert(MultiPoint multiPoint) {
+ return new org.wololo.geojson.MultiPoint(
+ convert(multiPoint.getCoordinates()));
+ }
+
+ org.wololo.geojson.LineString convert(LineString lineString) {
+ return new org.wololo.geojson.LineString(
+ convert(lineString.getCoordinates()));
+ }
+
+ org.wololo.geojson.MultiLineString convert(MultiLineString multiLineString) {
+ int size = multiLineString.getNumGeometries();
+ double[][][] lineStrings = new double[size][][];
+ for (int i = 0; i < size; i++) {
+ lineStrings[i] = convert(multiLineString.getGeometryN(i).getCoordinates());
+ }
+ return new org.wololo.geojson.MultiLineString(lineStrings);
+ }
+
+ org.wololo.geojson.Polygon convert(Polygon polygon) {
+ int size = polygon.getNumInteriorRing() + 1;
+ double[][][] rings = new double[size][][];
+ rings[0] = convert(polygon.getExteriorRing().getCoordinates());
+ for (int i = 0; i < size - 1; i++) {
+ rings[i + 1] = convert(polygon.getInteriorRingN(i).getCoordinates());
+ }
+ return new org.wololo.geojson.Polygon(rings);
+ }
+
+ org.wololo.geojson.MultiPolygon convert(MultiPolygon multiPolygon) {
+ int size = multiPolygon.getNumGeometries();
+ double[][][][] polygons = new double[size][][][];
+ for (int i = 0; i < size; i++) {
+ polygons[i] = convert((Polygon) multiPolygon.getGeometryN(i)).getCoordinates();
+ }
+ return new org.wololo.geojson.MultiPolygon(polygons);
+ }
+
+ org.wololo.geojson.GeometryCollection convert(GeometryCollection gc) {
+ int size = gc.getNumGeometries();
+ org.wololo.geojson.Geometry[] geometries = new org.wololo.geojson.Geometry[size];
+ for (int i = 0; i < size; i++) {
+ geometries[i] = write((Geometry) gc.getGeometryN(i));
+ }
+ return new org.wololo.geojson.GeometryCollection(geometries);
+ }
+
+ double[] convert(Coordinate coordinate) {
+ if(Double.isNaN( coordinate.getZ() )) {
+ return new double[] { coordinate.x, coordinate.y };
+ }
+ else {
+ return new double[] { coordinate.x, coordinate.y, coordinate.getZ() };
+ }
+ }
+
+ double[][] convert(Coordinate[] coordinates) {
+ double[][] array = new double[coordinates.length][];
+ for (int i = 0; i < coordinates.length; i++) {
+ array[i] = convert(coordinates[i]);
+ }
+ return array;
+ }
+}
diff --git a/core/src/main/java/org/apache/sedona/core/geometryObjects/SpatialIndexSerde.java b/core/src/main/java/org/apache/sedona/core/geometryObjects/SpatialIndexSerde.java
index 261867c..a8e5cbe 100644
--- a/core/src/main/java/org/apache/sedona/core/geometryObjects/SpatialIndexSerde.java
+++ b/core/src/main/java/org/apache/sedona/core/geometryObjects/SpatialIndexSerde.java
@@ -24,17 +24,10 @@ import com.esotericsoftware.kryo.Serializer;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import org.apache.log4j.Logger;
-import org.locationtech.jts.geom.Envelope;
-import org.locationtech.jts.geom.Geometry;
-import org.locationtech.jts.index.quadtree.Node;
+import org.locationtech.jts.index.quadtree.IndexSerde;
import org.locationtech.jts.index.quadtree.Quadtree;
-import org.locationtech.jts.index.strtree.AbstractNode;
-import org.locationtech.jts.index.strtree.ItemBoundable;
import org.locationtech.jts.index.strtree.STRtree;
-import java.util.ArrayList;
-import java.util.List;
-
/**
* Provides methods to efficiently serialize and deserialize spatialIndex types.
* <p>
@@ -69,51 +62,16 @@ public class SpatialIndexSerde
// serialize quadtree index
writeType(output, Type.QUADTREE);
Quadtree tree = (Quadtree) o;
- if (tree.isEmpty()) {
- output.writeByte(0);
- }
- else {
- output.writeByte(1);
- // write root
- List items = tree.getRoot().getItems();
- output.writeInt(items.size());
- for (Object item : items) {
- geometrySerde.write(kryo, output, item);
- }
- Node[] subNodes = tree.getRoot().getSubnode();
- for (int i = 0; i < 4; ++i) {
- writeQuadTreeNode(kryo, output, subNodes[i]);
- }
- }
+ IndexSerde indexSerde = new IndexSerde();
+ indexSerde.write(kryo, output, tree);
}
else if (o instanceof STRtree) {
//serialize rtree index
writeType(output, Type.RTREE);
STRtree tree = (STRtree) o;
- output.writeInt(tree.getNodeCapacity());
- if (tree.isEmpty()) {
- output.writeByte(0);
- }
- else {
- output.writeByte(1);
- // write head
- output.writeByte(tree.isBuilt() ? 1 : 0);
- if (!tree.isBuilt()) {
- // if not built, itemBoundables will not be null, record it
- ArrayList itemBoundables = tree.getItemBoundables();
- output.writeInt(itemBoundables.size());
- for (Object obj : itemBoundables) {
- if (!(obj instanceof ItemBoundable)) { throw new UnsupportedOperationException(" itemBoundables should only contain ItemBoundable objects "); }
- ItemBoundable itemBoundable = (ItemBoundable) obj;
- // write envelope
- writeItemBoundable(kryo, output, itemBoundable);
- }
- }
- else {
- // if built, write from root
- writeSTRTreeNode(kryo, output, tree.getRoot());
- }
- }
+ org.locationtech.jts.index.strtree.IndexSerde indexSerde
+ = new org.locationtech.jts.index.strtree.IndexSerde();
+ indexSerde.write(kryo, output, tree);
}
else {
throw new UnsupportedOperationException(" index type not supported ");
@@ -127,44 +85,13 @@ public class SpatialIndexSerde
Type indexType = Type.fromId(typeID);
switch (indexType) {
case QUADTREE: {
- Quadtree index = new Quadtree();
- boolean notEmpty = (input.readByte() & 0x01) == 1;
- if (!notEmpty) { return index; }
- int itemSize = input.readInt();
- List items = new ArrayList();
- for (int i = 0; i < itemSize; ++i) {
- items.add(geometrySerde.read(kryo, input, Geometry.class));
- }
- index.getRoot().setItems(items);
- for (int i = 0; i < 4; ++i) {
- index.getRoot().getSubnode()[i] = readQuadTreeNode(kryo, input);
- }
- return index;
+ IndexSerde indexSerde = new IndexSerde();
+ return indexSerde.read(kryo, input);
}
case RTREE: {
- int nodeCapacity = input.readInt();
- boolean notEmpty = (input.readByte() & 0x01) == 1;
- if (notEmpty) {
- STRtree index = new STRtree(nodeCapacity);
- boolean built = (input.readByte() & 0x01) == 1;
- if (built) {
- // if built, root is not null, set itemBoundables to null
- index.setBuilt(true);
- index.setItemBoundables(null);
- index.setRoot(readSTRtreeNode(kryo, input));
- }
- else {
- // if not built, just read itemBoundables
- ArrayList itemBoundables = new ArrayList();
- int itemSize = input.readInt();
- for (int i = 0; i < itemSize; ++i) {
- itemBoundables.add(readItemBoundable(kryo, input));
- }
- index.setItemBoundables(itemBoundables);
- }
- return index;
- }
- else { return new STRtree(nodeCapacity); }
+ org.locationtech.jts.index.strtree.IndexSerde indexSerde =
+ new org.locationtech.jts.index.strtree.IndexSerde();
+ return indexSerde.read(kryo, input);
}
default: {
throw new UnsupportedOperationException("can't deserialize spatial index of type" + indexType);
@@ -172,116 +99,6 @@ public class SpatialIndexSerde
}
}
- private void writeQuadTreeNode(Kryo kryo, Output output, Node node)
- {
- // write head first
- if (node == null || node.isEmpty()) {
- output.writeByte(0);
- }
- else { // not empty
- output.writeByte(1);
- // write node information, envelope and level
- geometrySerde.write(kryo, output, node.getEnvelope());
- output.writeInt(node.getLevel());
- List items = node.getItems();
- output.writeInt(items.size());
- for (Object obj : items) {
- geometrySerde.write(kryo, output, obj);
- }
- Node[] subNodes = node.getSubnode();
- for (int i = 0; i < 4; ++i) {
- writeQuadTreeNode(kryo, output, subNodes[i]);
- }
- }
- }
-
- private Node readQuadTreeNode(Kryo kryo, Input input)
- {
- boolean notEmpty = (input.readByte() & 0x01) == 1;
- if (!notEmpty) { return null; }
- Envelope envelope = (Envelope) geometrySerde.read(kryo, input, Envelope.class);
- int level = input.readInt();
- Node node = new Node(envelope, level);
- int itemSize = input.readInt();
- List items = new ArrayList();
- for (int i = 0; i < itemSize; ++i) {
- items.add(geometrySerde.read(kryo, input, Geometry.class));
- }
- node.setItems(items);
- // read children
- for (int i = 0; i < 4; ++i) {
- node.getSubnode()[i] = readQuadTreeNode(kryo, input);
- }
- return node;
- }
-
- private void writeSTRTreeNode(Kryo kryo, Output output, AbstractNode node)
- {
- // write head
- output.writeInt(node.getLevel());
- // write children
- List children = node.getChildBoundables();
- int childrenSize = children.size();
- output.writeInt(childrenSize);
- // if children not empty, write children
- if (childrenSize > 0) {
- if (children.get(0) instanceof AbstractNode) {
- // write type as 0, non-leaf node
- output.writeByte(0);
- for (Object obj : children) {
- AbstractNode child = (AbstractNode) obj;
- writeSTRTreeNode(kryo, output, child);
- }
- }
- else if (children.get(0) instanceof ItemBoundable) {
- // write type as 1, leaf node
- output.writeByte(1);
- // for leaf node, write items
- for (Object obj : children) {
- writeItemBoundable(kryo, output, (ItemBoundable) obj);
- }
- }
- else {
- throw new UnsupportedOperationException("wrong node type of STRtree");
- }
- }
- }
-
- private STRtree.STRtreeNode readSTRtreeNode(Kryo kryo, Input input)
- {
- int level = input.readInt();
- STRtree.STRtreeNode node = new STRtree.STRtreeNode(level);
- int childrenSize = input.readInt();
- boolean isLeaf = (input.readByte() & 0x01) == 1;
- ArrayList children = new ArrayList();
- if (isLeaf) {
- for (int i = 0; i < childrenSize; ++i) {
- children.add(readItemBoundable(kryo, input));
- }
- }
- else {
- for (int i = 0; i < childrenSize; ++i) {
- children.add(readSTRtreeNode(kryo, input));
- }
- }
- node.setChildBoundables(children);
- return node;
- }
-
- private void writeItemBoundable(Kryo kryo, Output output, ItemBoundable itemBoundable)
- {
- geometrySerde.write(kryo, output, itemBoundable.getBounds());
- geometrySerde.write(kryo, output, itemBoundable.getItem());
- }
-
- private ItemBoundable readItemBoundable(Kryo kryo, Input input)
- {
- return new ItemBoundable(
- geometrySerde.read(kryo, input, Envelope.class),
- geometrySerde.read(kryo, input, Geometry.class)
- );
- }
-
private void writeType(Output output, Type type)
{
output.writeByte((byte) type.id);
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 82fec48..ecdd90e 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
@@ -34,6 +34,7 @@ import org.apache.sedona.core.monitoring.Metric;
import org.apache.sedona.core.spatialPartitioning.SpatialPartitioner;
import org.apache.sedona.core.spatialRDD.CircleRDD;
import org.apache.sedona.core.spatialRDD.SpatialRDD;
+import org.apache.sedona.core.utils.GeomUtils;
import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
@@ -43,7 +44,9 @@ import org.apache.sedona.core.monitoring.Metrics;
import org.locationtech.jts.geom.Geometry;
import scala.Tuple2;
-import java.util.HashSet;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
import java.util.Objects;
public class JoinQuery
@@ -85,30 +88,17 @@ public class JoinQuery
}
}
- private static <U extends Geometry, T extends Geometry> JavaPairRDD<U, HashSet<T>> collectGeometriesByKey(JavaPairRDD<U, T> input)
+ private static <U extends Geometry, T extends Geometry> JavaPairRDD<U, List<T>> collectGeometriesByKey(JavaPairRDD<U, T> input)
{
- return input.aggregateByKey(
- new HashSet<T>(),
- new Function2<HashSet<T>, T, HashSet<T>>()
- {
- @Override
- public HashSet<T> call(HashSet<T> ts, T t)
- throws Exception
- {
- ts.add(t);
- return ts;
- }
- },
- new Function2<HashSet<T>, HashSet<T>, HashSet<T>>()
- {
- @Override
- public HashSet<T> call(HashSet<T> ts, HashSet<T> ts2)
- throws Exception
- {
- ts.addAll(ts2);
- return ts;
- }
- });
+ return input.groupBy(t -> GeomUtils.hashCode(t._1)).values().<U, List<T>>mapToPair(t -> {
+ List<T> values = new ArrayList<T>();
+ Iterator<Tuple2<U, T>> it = t.iterator();
+ Tuple2<U, T> firstTpl = it.next();
+ U key = firstTpl._1;
+ values.add(firstTpl._2);
+ while (it.hasNext()) { values.add(it.next()._2); }
+ return new Tuple2<U, List<T>>(key, values);
+ });
}
private static <U extends Geometry, T extends Geometry> JavaPairRDD<U, Long> countGeometriesByKey(JavaPairRDD<U, T> input)
@@ -148,9 +138,8 @@ public class JoinQuery
* 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>
- * Because the results are reported as a HashSet, any duplicates in the original spatialRDD will
- * be eliminated.
- *
+ * 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
@@ -160,42 +149,15 @@ public class JoinQuery
* @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, HashSet<T>> SpatialJoinQuery(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
+ 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
{
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection);
final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
return collectGeometriesByKey(joinResults);
}
- public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, HashSet<T>> SpatialJoinQuery(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, JoinParams joinParams)
- throws Exception
- {
- final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
- return collectGeometriesByKey(joinResults);
- }
-
- /**
- * A faster version of {@link #SpatialJoinQuery(SpatialRDD, SpatialRDD, boolean, boolean)} which may produce duplicate 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 considerBoundaryIntersection Join relationship type: 'intersects' if true, 'contains' otherwise
- * @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, HashSet<T>> SpatialJoinQueryWithDuplicates(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
- throws Exception
- {
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, true);
- final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
- return collectGeometriesByKey(joinResults);
- }
-
- public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, HashSet<T>> SpatialJoinQueryWithDuplicates(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, JoinParams joinParams)
+ public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, List<T>> SpatialJoinQuery(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, JoinParams joinParams)
throws Exception
{
final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
@@ -227,7 +189,7 @@ public class JoinQuery
public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, T> SpatialJoinQueryFlat(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
throws Exception
{
- final JoinParams params = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams params = new JoinParams(useIndex, considerBoundaryIntersection);
return spatialJoin(queryRDD, spatialRDD, params);
}
@@ -240,7 +202,8 @@ public class JoinQuery
/**
* {@link #SpatialJoinQueryFlat(SpatialRDD, SpatialRDD, boolean, boolean)} count by key.
* <p>
- * Duplicates present in the input RDDs will be reflected in the join results.
+ * 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
@@ -254,7 +217,7 @@ public class JoinQuery
public static <U extends Geometry, T extends Geometry> JavaPairRDD<U, Long> SpatialJoinQueryCountByKey(SpatialRDD<T> spatialRDD, SpatialRDD<U> queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
throws Exception
{
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection);
final JavaPairRDD<U, T> joinResults = spatialJoin(queryRDD, spatialRDD, joinParams);
return countGeometriesByKey(joinResults);
}
@@ -290,7 +253,7 @@ public class JoinQuery
public static <T extends Geometry> JavaPairRDD<Geometry, T> DistanceJoinQueryFlat(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
throws Exception
{
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection);
return distanceJoin(spatialRDD, queryRDD, joinParams);
}
@@ -313,9 +276,8 @@ public class JoinQuery
* 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>
- * Because the results are reported as a HashSet, any duplicates in the original spatialRDD will
- * be eliminated.
- *
+ * 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
@@ -324,41 +286,15 @@ public class JoinQuery
* @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, HashSet<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
+ public static <T extends Geometry> JavaPairRDD<Geometry, List<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
throws Exception
{
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection);
JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
return collectGeometriesByKey(joinResults);
}
- public static <T extends Geometry> JavaPairRDD<Geometry, HashSet<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, JoinParams joinParams)
- throws Exception
- {
- JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
- return collectGeometriesByKey(joinResults);
- }
-
- /**
- * A faster version of {@link #DistanceJoinQuery(SpatialRDD, CircleRDD, boolean, boolean)} which may produce duplicate 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 considerBoundaryIntersection consider boundary intersection
- * @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, HashSet<T>> DistanceJoinQueryWithDuplicates(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
- throws Exception
- {
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, true);
- JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
- return collectGeometriesByKey(joinResults);
- }
-
- public static <T extends Geometry> JavaPairRDD<Geometry, HashSet<T>> DistanceJoinQueryWithDuplicates(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, JoinParams joinParams)
+ public static <T extends Geometry> JavaPairRDD<Geometry, List<T>> DistanceJoinQuery(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, JoinParams joinParams)
throws Exception
{
JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
@@ -368,8 +304,8 @@ public class JoinQuery
/**
* {@link #DistanceJoinQueryFlat(SpatialRDD, CircleRDD, boolean, boolean)} count by key.
* <p>
- * Duplicates present in the input RDDs will be reflected in the join results.
- *
+ * 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
@@ -381,7 +317,7 @@ public class JoinQuery
public static <T extends Geometry> JavaPairRDD<Geometry, Long> DistanceJoinQueryCountByKey(SpatialRDD<T> spatialRDD, CircleRDD queryRDD, boolean useIndex, boolean considerBoundaryIntersection)
throws Exception
{
- final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection, false);
+ final JoinParams joinParams = new JoinParams(useIndex, considerBoundaryIntersection);
final JavaPairRDD<Geometry, T> joinResults = distanceJoin(spatialRDD, queryRDD, joinParams);
return countGeometriesByKey(joinResults);
}
@@ -438,17 +374,17 @@ public class JoinQuery
(SpatialPartitioner) rightRDD.spatialPartitionedRDD.partitioner().get();
final DedupParams dedupParams = partitioner.getDedupParams();
- final JavaRDD<Pair<U, T>> resultWithDuplicates;
+ final JavaRDD<Pair<U, T>> joinResult;
if (joinParams.useIndex) {
if (rightRDD.indexedRDD != null) {
final RightIndexLookupJudgement judgement =
new RightIndexLookupJudgement(joinParams.considerBoundaryIntersection, dedupParams);
- resultWithDuplicates = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.indexedRDD, judgement);
+ joinResult = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.indexedRDD, judgement);
}
else if (leftRDD.indexedRDD != null) {
final LeftIndexLookupJudgement judgement =
new LeftIndexLookupJudgement(joinParams.considerBoundaryIntersection, dedupParams);
- resultWithDuplicates = leftRDD.indexedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
+ joinResult = leftRDD.indexedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
}
else {
log.warn("UseIndex is true, but no index exists. Will build index on the fly.");
@@ -459,31 +395,15 @@ public class JoinQuery
joinParams.joinBuildSide,
dedupParams,
buildCount, streamCount, resultCount, candidateCount);
- resultWithDuplicates = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
+ joinResult = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
}
- }/*
- else if (joinParams.indexType != null) {
- DynamicIndexLookupJudgement judgement =
- new DynamicIndexLookupJudgement(
- joinParams.considerBoundaryIntersection,
- joinParams.indexType,
- joinParams.joinBuildSide,
- dedupParams,
- buildCount, streamCount, resultCount, candidateCount);
- resultWithDuplicates = leftRDD.spatialPartitionedRDD.zipPartitions(rightRDD.spatialPartitionedRDD, judgement);
- }*/
+ }
else {
NestedLoopJudgement judgement = new NestedLoopJudgement(joinParams.considerBoundaryIntersection, dedupParams);
- resultWithDuplicates = rightRDD.spatialPartitionedRDD.zipPartitions(leftRDD.spatialPartitionedRDD, judgement);
+ joinResult = rightRDD.spatialPartitionedRDD.zipPartitions(leftRDD.spatialPartitionedRDD, judgement);
}
- final boolean uniqueResults = dedupParams != null;
-
- final JavaRDD<Pair<U, T>> result =
- (joinParams.allowDuplicates || uniqueResults) ? resultWithDuplicates
- : resultWithDuplicates.distinct();
-
- return result.mapToPair(new PairFunction<Pair<U, T>, U, T>()
+ return joinResult.mapToPair(new PairFunction<Pair<U, T>, U, T>()
{
@Override
public Tuple2<U, T> call(Pair<U, T> pair)
@@ -498,15 +418,13 @@ public class JoinQuery
{
public final boolean useIndex;
public final boolean considerBoundaryIntersection;
- public final boolean allowDuplicates;
public final IndexType indexType;
public final JoinBuildSide joinBuildSide;
- public JoinParams(boolean useIndex, boolean considerBoundaryIntersection, boolean allowDuplicates)
+ public JoinParams(boolean useIndex, boolean considerBoundaryIntersection)
{
this.useIndex = useIndex;
this.considerBoundaryIntersection = considerBoundaryIntersection;
- this.allowDuplicates = allowDuplicates;
this.indexType = IndexType.RTREE;
this.joinBuildSide = JoinBuildSide.RIGHT;
}
@@ -515,7 +433,6 @@ public class JoinQuery
{
this.useIndex = false;
this.considerBoundaryIntersection = considerBoundaryIntersection;
- this.allowDuplicates = false;
this.indexType = polygonIndexType;
this.joinBuildSide = joinBuildSide;
}
diff --git a/core/src/main/java/org/apache/sedona/core/spatialPartitioning/RtreePartitioning.java b/core/src/main/java/org/apache/sedona/core/spatialPartitioning/RtreePartitioning.java
index bace1ff..d015dfb 100644
--- a/core/src/main/java/org/apache/sedona/core/spatialPartitioning/RtreePartitioning.java
+++ b/core/src/main/java/org/apache/sedona/core/spatialPartitioning/RtreePartitioning.java
@@ -20,6 +20,8 @@
package org.apache.sedona.core.spatialPartitioning;
import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.index.strtree.AbstractNode;
+import org.locationtech.jts.index.strtree.Boundable;
import org.locationtech.jts.index.strtree.STRtree;
import java.io.Serializable;
@@ -54,8 +56,7 @@ public class RtreePartitioning
for (Envelope sample : samples) {
strtree.insert(sample, sample);
}
-
- List<Envelope> envelopes = strtree.findLeafBounds();
+ List<Envelope> envelopes = findLeafBounds(strtree);
for (Envelope envelope : envelopes) {
grids.add(envelope);
}
@@ -71,4 +72,48 @@ public class RtreePartitioning
return this.grids;
}
+
+ /**
+ * This function traverses the boundaries of all leaf nodes.
+ * This function should be called after all insertions.
+ * @param stRtree
+ * @return The list of leaf nodes boundaries
+ */
+ private List findLeafBounds(STRtree stRtree){
+ stRtree.build();
+ List boundaries = new ArrayList();
+ if (stRtree.isEmpty()) {
+ //Assert.isTrue(root.getBounds() == null);
+ //If the root is empty, we stop traversing. This should not happen.
+ return boundaries;
+ }
+ findLeafBounds(stRtree.getRoot(), boundaries);
+ return boundaries;
+ }
+
+ private void findLeafBounds(AbstractNode node, List boundaries) {
+ List childBoundables = node.getChildBoundables();
+ boolean flagLeafnode=true;
+ for (Object boundable : childBoundables) {
+ Boundable childBoundable = (Boundable) boundable;
+ if (childBoundable instanceof AbstractNode) {
+ //We find this is not a leaf node.
+ flagLeafnode = false;
+ break;
+ }
+ }
+ if(flagLeafnode)
+ {
+ boundaries.add(node.getBounds());
+ }
+ else
+ {
+ for (Object boundable : childBoundables) {
+ Boundable childBoundable = (Boundable) boundable;
+ if (childBoundable instanceof AbstractNode) {
+ findLeafBounds((AbstractNode) childBoundable, boundaries);
+ }
+ }
+ }
+ }
}
diff --git a/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java b/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
index d076c22..dd93b5c 100644
--- a/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
+++ b/core/src/main/java/org/apache/sedona/core/spatialRDD/SpatialRDD.java
@@ -23,15 +23,12 @@ import org.apache.commons.lang.NullArgumentException;
import org.apache.log4j.Logger;
import org.apache.sedona.core.enums.GridType;
import org.apache.sedona.core.enums.IndexType;
-import org.apache.sedona.core.spatialPartitioning.EqualPartitioning;
+import org.apache.sedona.core.geometryObjects.GeoJSONWriterNew;
import org.apache.sedona.core.spatialPartitioning.FlatGridPartitioner;
-import org.apache.sedona.core.spatialPartitioning.HilbertPartitioning;
import org.apache.sedona.core.spatialPartitioning.KDBTree;
import org.apache.sedona.core.spatialPartitioning.KDBTreePartitioner;
import org.apache.sedona.core.spatialPartitioning.QuadtreePartitioning;
-import org.apache.sedona.core.spatialPartitioning.RtreePartitioning;
import org.apache.sedona.core.spatialPartitioning.SpatialPartitioner;
-import org.apache.sedona.core.spatialPartitioning.VoronoiPartitioning;
import org.apache.sedona.core.spatialPartitioning.quadtree.QuadTreePartitioner;
import org.apache.sedona.core.spatialPartitioning.quadtree.StandardQuadTree;
import org.apache.sedona.core.spatialRddTool.IndexBuilder;
@@ -59,7 +56,6 @@ import org.opengis.referencing.FactoryException;
import org.opengis.referencing.crs.CoordinateReferenceSystem;
import org.opengis.referencing.operation.MathTransform;
import org.wololo.geojson.Feature;
-import org.wololo.jts2geojson.GeoJSONWriter;
import scala.Tuple2;
import java.io.Serializable;
@@ -114,12 +110,6 @@ public class SpatialRDD<T extends Geometry>
*/
public JavaRDD<T> rawSpatialRDD;
- /**
- * The grids.
- */
- public List<Envelope> grids;
-
- public StandardQuadTree partitionTree;
public List<String> fieldNames;
/**
* The CR stransformation.
@@ -262,34 +252,9 @@ public class SpatialRDD<T extends Geometry>
boundaryEnvelope.getMinY(), boundaryEnvelope.getMaxY() + 0.01);
switch (gridType) {
- case EQUALGRID: {
- EqualPartitioning EqualPartitioning = new EqualPartitioning(paddedBoundary, numPartitions);
- grids = EqualPartitioning.getGrids();
- partitioner = new FlatGridPartitioner(grids);
- break;
- }
- case HILBERT: {
- HilbertPartitioning hilbertPartitioning = new HilbertPartitioning(samples, paddedBoundary, numPartitions);
- grids = hilbertPartitioning.getGrids();
- partitioner = new FlatGridPartitioner(grids);
- break;
- }
- case RTREE: {
- RtreePartitioning rtreePartitioning = new RtreePartitioning(samples, numPartitions);
- grids = rtreePartitioning.getGrids();
- partitioner = new FlatGridPartitioner(grids);
- break;
- }
- case VORONOI: {
- VoronoiPartitioning voronoiPartitioning = new VoronoiPartitioning(samples, numPartitions);
- grids = voronoiPartitioning.getGrids();
- partitioner = new FlatGridPartitioner(grids);
- break;
- }
case QUADTREE: {
QuadtreePartitioning quadtreePartitioning = new QuadtreePartitioning(samples, paddedBoundary, numPartitions);
- partitionTree = quadtreePartitioning.getPartitionTree();
- partitioner = new QuadTreePartitioner(partitionTree);
+ partitioner = new QuadTreePartitioner(quadtreePartitioning.getPartitionTree());
break;
}
case KDBTREE: {
@@ -302,7 +267,8 @@ public class SpatialRDD<T extends Geometry>
break;
}
default:
- throw new Exception("[AbstractSpatialRDD][spatialPartitioning] Unsupported spatial partitioning method.");
+ throw new Exception("[AbstractSpatialRDD][spatialPartitioning] Unsupported spatial partitioning method. " +
+ "The following partitioning methods are not longer supported: R-Tree, Hilbert curve, Voronoi, Equal-Grids");
}
this.spatialPartitionedRDD = partition(partitioner);
@@ -327,7 +293,6 @@ public class SpatialRDD<T extends Geometry>
{
this.partitioner = new FlatGridPartitioner(otherGrids);
this.spatialPartitionedRDD = partition(partitioner);
- this.grids = otherGrids;
return true;
}
@@ -339,7 +304,6 @@ public class SpatialRDD<T extends Geometry>
{
this.partitioner = new QuadTreePartitioner(partitionTree);
this.spatialPartitionedRDD = partition(partitioner);
- this.partitionTree = partitionTree;
return true;
}
@@ -614,7 +578,7 @@ public class SpatialRDD<T extends Geometry>
throws Exception
{
ArrayList<String> result = new ArrayList();
- GeoJSONWriter writer = new GeoJSONWriter();
+ GeoJSONWriterNew writer = new GeoJSONWriterNew();
while (iterator.hasNext()) {
Geometry spatialObject = iterator.next();
Feature jsonFeature;
diff --git a/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java b/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java
new file mode 100644
index 0000000..30c22ee
--- /dev/null
+++ b/core/src/main/java/org/apache/sedona/core/utils/GeomUtils.java
@@ -0,0 +1,46 @@
+/**
+ * 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.utils;
+
+import org.locationtech.jts.geom.Geometry;
+
+import java.util.Objects;
+
+public class GeomUtils
+{
+ public static String printGeom(Geometry geom) {
+ if(geom.getUserData()!=null) return geom.toText() + "\t" + geom.getUserData();
+ else return geom.toText();
+ }
+
+ public static String printGeom(Object geom) {
+ Geometry g = (Geometry) geom;
+ return printGeom(g);
+ }
+
+ public static int hashCode(Geometry geom) {
+ return geom.getUserData()==null? geom.hashCode():geom.hashCode()*31 + geom.getUserData().hashCode();
+ }
+ public static boolean equalsTopoGeom(Geometry geom1, Geometry geom2) {
+ if (Objects.equals(geom1.getUserData(), geom2.getUserData())) return geom1.equals(geom2);
+ return false;
+ }
+
+ public static boolean equalsExactGeom(Geometry geom1, Object geom2) {
+ if (! (geom2 instanceof Geometry)) return false;
+ Geometry g = (Geometry) geom2;
+ if (Objects.equals(geom1.getUserData(), g.getUserData())) return geom1.equalsExact(g);
+ else return false;
+ }
+}
diff --git a/core/src/main/java/org/apache/sedona/core/utils/SedonaConf.java b/core/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
index 6f3f52e..1b8b2b7 100644
--- a/core/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
+++ b/core/src/main/java/org/apache/sedona/core/utils/SedonaConf.java
@@ -51,7 +51,7 @@ public class SedonaConf
private Integer fallbackPartitionNum = -1;
- private GridType joinGridType = GridType.QUADTREE;
+ private GridType joinGridType = GridType.KDBTREE;
public SedonaConf(SparkConf sparkConf)
{
diff --git a/core/src/main/java/org/locationtech/jts/index/quadtree/IndexSerde.java b/core/src/main/java/org/locationtech/jts/index/quadtree/IndexSerde.java
new file mode 100644
index 0000000..5dd6556
--- /dev/null
+++ b/core/src/main/java/org/locationtech/jts/index/quadtree/IndexSerde.java
@@ -0,0 +1,115 @@
+/**
+ * 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.locationtech.jts.index.quadtree;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.sedona.core.geometryObjects.GeometrySerde;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides methods to efficiently serialize and deserialize the index.
+ * trees are serialized recursively.
+ */
+public class IndexSerde
+{
+ GeometrySerde geometrySerde;
+ public IndexSerde() {
+ geometrySerde = new GeometrySerde();
+ }
+
+ public Object read(Kryo kryo, Input input){
+ Quadtree index = new Quadtree();
+ boolean notEmpty = (input.readByte() & 0x01) == 1;
+ if (!notEmpty) { return index; }
+ int itemSize = input.readInt();
+ List items = new ArrayList();
+ for (int i = 0; i < itemSize; ++i) {
+ items.add(geometrySerde.read(kryo, input, Geometry.class));
+ }
+ index.getRoot().items = items;
+ for (int i = 0; i < 4; ++i) {
+ index.getRoot().subnode[i] = readQuadTreeNode(kryo, input);
+ }
+ return index;
+ }
+
+ public void write(Kryo kryo, Output output, Quadtree tree) {
+ // serialize quadtree index
+ if (tree.isEmpty()) {
+ output.writeByte(0);
+ }
+ else {
+ output.writeByte(1);
+ // write root
+ List items = tree.getRoot().getItems();
+ output.writeInt(items.size());
+ for (Object item : items) {
+ geometrySerde.write(kryo, output, item);
+ }
+ Node[] subNodes = tree.getRoot().subnode;
+ for (int i = 0; i < 4; ++i) {
+ writeQuadTreeNode(kryo, output, subNodes[i]);
+ }
+ }
+ }
+
+ private void writeQuadTreeNode(Kryo kryo, Output output, Node node)
+ {
+ // write head first
+ if (node == null || node.isEmpty()) {
+ output.writeByte(0);
+ }
+ else { // not empty
+ output.writeByte(1);
+ // write node information, envelope and level
+ geometrySerde.write(kryo, output, node.getEnvelope());
+ output.writeInt(node.getLevel());
+ List items = node.getItems();
+ output.writeInt(items.size());
+ for (Object obj : items) {
+ geometrySerde.write(kryo, output, obj);
+ }
+ Node[] subNodes = node.subnode;
+ for (int i = 0; i < 4; ++i) {
+ writeQuadTreeNode(kryo, output, subNodes[i]);
+ }
+ }
+ }
+
+ private Node readQuadTreeNode(Kryo kryo, Input input)
+ {
+ boolean notEmpty = (input.readByte() & 0x01) == 1;
+ if (!notEmpty) { return null; }
+ Envelope envelope = (Envelope) geometrySerde.read(kryo, input, Envelope.class);
+ int level = input.readInt();
+ Node node = new Node(envelope, level);
+ int itemSize = input.readInt();
+ List items = new ArrayList();
+ for (int i = 0; i < itemSize; ++i) {
+ items.add(geometrySerde.read(kryo, input, Geometry.class));
+ }
+ node.items = items;
+ // read children
+ for (int i = 0; i < 4; ++i) {
+ node.subnode[i] = readQuadTreeNode(kryo, input);
+ }
+ return node;
+ }
+}
diff --git a/core/src/main/java/org/locationtech/jts/index/strtree/IndexSerde.java b/core/src/main/java/org/locationtech/jts/index/strtree/IndexSerde.java
new file mode 100644
index 0000000..bf3d308
--- /dev/null
+++ b/core/src/main/java/org/locationtech/jts/index/strtree/IndexSerde.java
@@ -0,0 +1,155 @@
+/**
+ * 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.locationtech.jts.index.strtree;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.sedona.core.geometryObjects.GeometrySerde;
+import org.locationtech.jts.geom.Envelope;
+import org.locationtech.jts.geom.Geometry;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Provides methods to efficiently serialize and deserialize the index.
+ * trees are serialized recursively.
+ */
+public class IndexSerde
+{
+ GeometrySerde geometrySerde;
+ public IndexSerde() {
+ geometrySerde = new GeometrySerde();
+ }
+
+ public Object read(Kryo kryo, Input input){
+ int nodeCapacity = input.readInt();
+ boolean notEmpty = (input.readByte() & 0x01) == 1;
+ if (notEmpty) {
+ boolean built = (input.readByte() & 0x01) == 1;
+ if (built) {
+ // if built, root is not null, set itemBoundables to null
+ STRtree index = new STRtree(nodeCapacity, readSTRtreeNode(kryo, input));
+ return index;
+ }
+ else {
+ // if not built, just read itemBoundables
+ ArrayList itemBoundables = new ArrayList();
+ int itemSize = input.readInt();
+ for (int i = 0; i < itemSize; ++i) {
+ itemBoundables.add(readItemBoundable(kryo, input));
+ }
+ STRtree index = new STRtree(nodeCapacity, itemBoundables);
+ return index;
+ }
+ }
+ else { return new STRtree(nodeCapacity); }
+ }
+
+ public void write(Kryo kryo, Output output, STRtree tree) {
+ output.writeInt(tree.getNodeCapacity());
+ if (tree.isEmpty()) {
+ output.writeByte(0);
+ }
+ else {
+ output.writeByte(1);
+ // write head
+ boolean isBuilt = tree.getItemBoundables() == null;
+ output.writeByte(isBuilt ? 1 : 0);
+ if (!isBuilt) {
+ // if not built, itemBoundables will not be null, record it
+ ArrayList itemBoundables = tree.getItemBoundables();
+ output.writeInt(itemBoundables.size());
+ for (Object obj : itemBoundables) {
+ if (!(obj instanceof ItemBoundable)) { throw new UnsupportedOperationException(" itemBoundables should only contain ItemBoundable objects "); }
+ ItemBoundable itemBoundable = (ItemBoundable) obj;
+ // write envelope
+ writeItemBoundable(kryo, output, itemBoundable);
+ }
+ }
+ else {
+ // if built, write from root
+ writeSTRTreeNode(kryo, output, tree.getRoot());
+ }
+ }
+ }
+
+ private void writeSTRTreeNode(Kryo kryo, Output output, AbstractNode node)
+ {
+ // write head
+ output.writeInt(node.getLevel());
+ // write children
+ List children = node.getChildBoundables();
+ int childrenSize = children.size();
+ output.writeInt(childrenSize);
+ // if children not empty, write children
+ if (childrenSize > 0) {
+ if (children.get(0) instanceof AbstractNode) {
+ // write type as 0, non-leaf node
+ output.writeByte(0);
+ for (Object obj : children) {
+ AbstractNode child = (AbstractNode) obj;
+ writeSTRTreeNode(kryo, output, child);
+ }
+ }
+ else if (children.get(0) instanceof ItemBoundable) {
+ // write type as 1, leaf node
+ output.writeByte(1);
+ // for leaf node, write items
+ for (Object obj : children) {
+ writeItemBoundable(kryo, output, (ItemBoundable) obj);
+ }
+ }
+ else {
+ throw new UnsupportedOperationException("wrong node type of STRtree");
+ }
+ }
+ }
+
+ private STRtree.STRtreeNode readSTRtreeNode(Kryo kryo, Input input)
+ {
+ int level = input.readInt();
+ STRtree.STRtreeNode node = new STRtree.STRtreeNode(level);
+ int childrenSize = input.readInt();
+ boolean isLeaf = (input.readByte() & 0x01) == 1;
+ ArrayList children = new ArrayList();
+ if (isLeaf) {
+ for (int i = 0; i < childrenSize; ++i) {
+ children.add(readItemBoundable(kryo, input));
+ }
+ }
+ else {
+ for (int i = 0; i < childrenSize; ++i) {
+ children.add(readSTRtreeNode(kryo, input));
+ }
+ }
+ node.setChildBoundables(children);
+ return node;
+ }
+
+ private void writeItemBoundable(Kryo kryo, Output output, ItemBoundable itemBoundable)
+ {
+ geometrySerde.write(kryo, output, itemBoundable.getBounds());
+ geometrySerde.write(kryo, output, itemBoundable.getItem());
+ }
+
+ private ItemBoundable readItemBoundable(Kryo kryo, Input input)
+ {
+ return new ItemBoundable(
+ geometrySerde.read(kryo, input, Envelope.class),
+ geometrySerde.read(kryo, input, Geometry.class)
+ );
+ }
+}
diff --git a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/GeometrySerdeTest.java b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/GeometrySerdeTest.java
index 8db4703..3bb9af4 100644
--- a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/GeometrySerdeTest.java
+++ b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/GeometrySerdeTest.java
@@ -24,7 +24,7 @@ import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import org.apache.sedona.core.geometryObjects.Circle;
import org.apache.sedona.core.geometryObjects.GeometrySerde;
-import org.junit.Assert;
+import org.apache.sedona.core.utils.GeomUtils;
import org.junit.Test;
import org.locationtech.jts.geom.Geometry;
import org.locationtech.jts.geom.GeometryCollection;
@@ -33,6 +33,8 @@ import org.locationtech.jts.io.WKTReader;
import java.io.ByteArrayOutputStream;
+import static org.junit.Assert.assertTrue;
+
public class GeometrySerdeTest
{
private final Kryo kryo = new Kryo();
@@ -57,17 +59,17 @@ public class GeometrySerdeTest
throws Exception
{
Geometry geometry = parseWkt(wkt);
- Assert.assertEquals(geometry, serde(geometry));
+ assertTrue(GeomUtils.equalsExactGeom(geometry, serde(geometry)));
geometry.setUserData("This is a test");
- Assert.assertEquals(geometry, serde(geometry));
+ assertTrue(GeomUtils.equalsExactGeom(geometry, serde(geometry)));
if (geometry instanceof GeometryCollection) {
return;
}
Circle circle = new Circle(geometry, 1.2);
- Assert.assertEquals(circle, serde(circle));
+ assertTrue(GeomUtils.equalsExactGeom(circle, serde(circle)));
}
private Geometry parseWkt(String wkt)
diff --git a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileRDDTest.java b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileRDDTest.java
index e401e1d..c0bf3c9 100644
--- a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileRDDTest.java
+++ b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileRDDTest.java
@@ -26,6 +26,7 @@ import org.apache.sedona.core.spatialOperator.RangeQuery;
import org.apache.sedona.core.spatialRDD.LineStringRDD;
import org.apache.sedona.core.spatialRDD.PointRDD;
import org.apache.sedona.core.spatialRDD.PolygonRDD;
+import org.apache.sedona.core.utils.GeomUtils;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.geotools.data.DataStore;
@@ -143,7 +144,7 @@ public class ShapefileRDDTest
geometry = multiPolygon.getGeometryN(0);
}
}
- featureTexts.add(String.valueOf(geometry));
+ featureTexts.add(GeomUtils.printGeom(geometry));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -186,7 +187,7 @@ public class ShapefileRDDTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -229,7 +230,7 @@ public class ShapefileRDDTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -264,7 +265,7 @@ public class ShapefileRDDTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -314,7 +315,7 @@ public class ShapefileRDDTest
geometry = multiPolygon.getGeometryN(0);
}
}
- featureTexts.add(String.valueOf(geometry));
+ featureTexts.add(GeomUtils.printGeom(geometry));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
diff --git a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileReaderTest.java b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileReaderTest.java
index d2132f4..78eb8d9 100644
--- a/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileReaderTest.java
+++ b/core/src/test/java/org/apache/sedona/core/formatMapper/shapefileParser/shapes/ShapefileReaderTest.java
@@ -34,6 +34,7 @@ import org.apache.sedona.core.spatialRDD.LineStringRDD;
import org.apache.sedona.core.spatialRDD.PointRDD;
import org.apache.sedona.core.spatialRDD.PolygonRDD;
import org.apache.sedona.core.spatialRDD.SpatialRDD;
+import org.apache.sedona.core.utils.GeomUtils;
import org.geotools.data.DataStore;
import org.geotools.data.DataStoreFinder;
import org.geotools.data.FeatureSource;
@@ -161,7 +162,7 @@ public class ShapefileReaderTest
geometry = multiPolygon.getGeometryN(0);
}
}
- featureTexts.add(String.valueOf(geometry));
+ featureTexts.add(GeomUtils.printGeom(geometry));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -193,7 +194,7 @@ public class ShapefileReaderTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -223,7 +224,7 @@ public class ShapefileReaderTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
@@ -253,7 +254,7 @@ public class ShapefileReaderTest
ArrayList<String> featureTexts = new ArrayList<String>();
while (features.hasNext()) {
SimpleFeature feature = features.next();
- featureTexts.add(String.valueOf(feature.getDefaultGeometry()));
+ featureTexts.add(GeomUtils.printGeom(feature.getDefaultGeometry()));
}
features.close();
final Iterator<String> featureIterator = featureTexts.iterator();
diff --git a/core/src/test/java/org/apache/sedona/core/geometryObjects/CircleTest.java b/core/src/test/java/org/apache/sedona/core/geometryObjects/CircleTest.java
index c767058..7ef7c3c 100644
--- a/core/src/test/java/org/apache/sedona/core/geometryObjects/CircleTest.java
+++ b/core/src/test/java/org/apache/sedona/core/geometryObjects/CircleTest.java
@@ -18,6 +18,7 @@
*/
package org.apache.sedona.core.geometryObjects;
+import org.apache.sedona.core.utils.GeomUtils;
import org.junit.Test;
import org.locationtech.jts.geom.Coordinate;
import org.locationtech.jts.geom.Envelope;
@@ -29,7 +30,6 @@ import org.locationtech.jts.io.WKTReader;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
// TODO: Auto-generated Javadoc
@@ -206,13 +206,13 @@ public class CircleTest
@Test
public void testEquality()
{
- assertEquals(
+ assertTrue(GeomUtils.equalsExactGeom(
new Circle(makePoint(-112.574945, 45.987772), 0.01),
- new Circle(makePoint(-112.574945, 45.987772), 0.01));
+ new Circle(makePoint(-112.574945, 45.987772), 0.01)));
- assertNotEquals(
+ assertFalse(GeomUtils.equalsExactGeom(
new Circle(makePoint(-112.574945, 45.987772), 0.01),
- new Circle(makePoint(-112.574942, 45.987772), 0.01));
+ new Circle(makePoint(-112.574942, 45.987772), 0.01)));
}
private Point makePoint(double x, double y)
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryCorrectnessChecker.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryCorrectnessChecker.java
index f92ee4f..9959f31 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryCorrectnessChecker.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinQueryCorrectnessChecker.java
@@ -107,7 +107,6 @@ public class JoinQueryCorrectnessChecker
public static Collection testParams()
{
return Arrays.asList(new Object[][] {
- {GridType.RTREE},
{GridType.QUADTREE},
{GridType.KDBTREE},
});
@@ -247,10 +246,10 @@ public class JoinQueryCorrectnessChecker
PointRDD objectRDD = new PointRDD(sc.parallelize(testInsidePointSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
verifyJoinResults(resultNoIndex);
}
@@ -267,10 +266,10 @@ public class JoinQueryCorrectnessChecker
PointRDD objectRDD = new PointRDD(sc.parallelize(testOnBoundaryPointSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
verifyJoinResults(resultNoIndex);
}
@@ -287,10 +286,10 @@ public class JoinQueryCorrectnessChecker
PointRDD objectRDD = new PointRDD(sc.parallelize(testOutsidePointSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
assertEquals(0, result.size());
- List<Tuple2<Polygon, HashSet<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<Point>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
assertEquals(0, resultNoIndex.size());
}
@@ -307,10 +306,10 @@ public class JoinQueryCorrectnessChecker
LineStringRDD objectRDD = new LineStringRDD(sc.parallelize(testInsideLineStringSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
verifyJoinResults(resultNoIndex);
}
@@ -327,10 +326,10 @@ public class JoinQueryCorrectnessChecker
LineStringRDD objectRDD = new LineStringRDD(sc.parallelize(testOverlappedLineStringSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, true).collect();
+ List<Tuple2<Polygon, List<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, true).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, true).collect();
+ List<Tuple2<Polygon, List<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, true).collect();
verifyJoinResults(resultNoIndex);
}
@@ -347,10 +346,10 @@ public class JoinQueryCorrectnessChecker
LineStringRDD objectRDD = new LineStringRDD(sc.parallelize(testOutsideLineStringSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<LineString>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
assertEquals(0, result.size());
- List<Tuple2<Polygon, HashSet<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<LineString>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
assertEquals(0, resultNoIndex.size());
}
@@ -367,10 +366,10 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testInsidePolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Polygon, List<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
verifyJoinResults(resultNoIndex);
}
@@ -387,19 +386,19 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testOverlappedPolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, true).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, true).collect();
verifyJoinResults(result);
- List<Tuple2<Polygon, HashSet<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, true).collect();
verifyJoinResults(resultNoIndex);
}
- private <U extends Geometry, T extends Geometry> void verifyJoinResults(List<Tuple2<U, HashSet<T>>> result)
+ private <U extends Geometry, T extends Geometry> void verifyJoinResults(List<Tuple2<U, List<T>>> result)
{
assertEquals(200, result.size());
- for (Tuple2<U, HashSet<T>> tuple : result) {
+ for (Tuple2<U, List<T>> tuple : result) {
U window = tuple._1;
- Set<T> objects = tuple._2;
+ List<T> objects = tuple._2;
String windowUserData = (String) window.getUserData();
String[] tokens = windowUserData.split(":", 2);
@@ -431,11 +430,11 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testOutsidePolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, true, false).collect();
assertEquals(0, result.size());
- List<Tuple2<Polygon, HashSet<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
- assertEquals(0, result.size());
+ List<Tuple2<Polygon, List<Polygon>>> resultNoIndex = JoinQuery.SpatialJoinQuery(objectRDD, windowRDD, false, false).collect();
+ assertEquals(0, resultNoIndex.size());
}
/**
@@ -452,10 +451,10 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testInsidePolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Geometry, HashSet<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Geometry, List<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, false).collect();
verifyJoinResults(result);
- List<Tuple2<Geometry, HashSet<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, false).collect();
+ List<Tuple2<Geometry, List<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, false).collect();
verifyJoinResults(resultNoIndex);
}
@@ -473,10 +472,10 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testOverlappedPolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Geometry, HashSet<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, true).collect();
+ List<Tuple2<Geometry, List<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, true).collect();
verifyJoinResults(result);
- List<Tuple2<Geometry, HashSet<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, true).collect();
+ List<Tuple2<Geometry, List<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, true).collect();
verifyJoinResults(resultNoIndex);
}
@@ -494,10 +493,10 @@ public class JoinQueryCorrectnessChecker
PolygonRDD objectRDD = new PolygonRDD(sc.parallelize(testOutsidePolygonSet), StorageLevel.MEMORY_ONLY());
prepareRDDs(objectRDD, windowRDD);
- List<Tuple2<Geometry, HashSet<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, true).collect();
+ List<Tuple2<Geometry, List<Polygon>>> result = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, true).collect();
assertEquals(0, result.size());
- List<Tuple2<Geometry, HashSet<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, true).collect();
+ List<Tuple2<Geometry, List<Polygon>>> resultNoIndex = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, false, true).collect();
assertEquals(0, resultNoIndex.size());
}
}
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinTestBase.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinTestBase.java
index f641bbe..53173eb 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinTestBase.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/JoinTestBase.java
@@ -34,12 +34,10 @@ import scala.Tuple2;
import java.io.IOException;
import java.io.InputStream;
-import java.util.HashSet;
import java.util.List;
import java.util.Properties;
import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
class JoinTestBase
@@ -88,12 +86,9 @@ class JoinTestBase
protected final GridType gridType;
- protected final boolean useLegacyPartitionAPIs;
-
- protected JoinTestBase(GridType gridType, boolean useLegacyPartitionAPIs, int numPartitions)
+ protected JoinTestBase(GridType gridType, int numPartitions)
{
this.gridType = gridType;
- this.useLegacyPartitionAPIs = useLegacyPartitionAPIs;
JoinTestBase.numPartitions = numPartitions;
}
@@ -162,17 +157,7 @@ class JoinTestBase
throws Exception
{
spatialRDD.spatialPartitioning(gridType);
- if (useLegacyPartitionAPIs) {
- if (gridType != GridType.QUADTREE) {
- queryRDD.spatialPartitioning(spatialRDD.grids);
- }
- else {
- queryRDD.spatialPartitioning(spatialRDD.partitionTree);
- }
- }
- else {
- queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
- }
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
}
protected boolean expectToPreserveOriginalDuplicates()
@@ -180,22 +165,20 @@ class JoinTestBase
return gridType == GridType.QUADTREE || gridType == GridType.KDBTREE;
}
- protected <T extends Geometry> long countJoinResults(List<Tuple2<Polygon, HashSet<T>>> results)
+ protected <T extends Geometry> long countJoinResults(List<Tuple2<Polygon, List<T>>> results)
{
int count = 0;
- for (final Tuple2<Polygon, HashSet<T>> tuple : results) {
+ for (final Tuple2<Polygon, List<T>> tuple : results) {
count += tuple._2().size();
}
return count;
}
- protected <T extends Geometry> void sanityCheckJoinResults(List<Tuple2<Polygon, HashSet<T>>> results)
+ protected <T extends Geometry> void sanityCheckJoinResults(List<Tuple2<Polygon, List<T>>> results)
{
- for (final Tuple2<Polygon, HashSet<T>> tuple : results) {
- assertNotNull(tuple._1().getUserData());
+ for (final Tuple2<Polygon, List<T>> tuple : results) {
assertFalse(tuple._2().isEmpty());
for (final T shape : tuple._2()) {
- assertNotNull(shape.getUserData());
assertTrue(tuple._1().intersects(shape));
}
}
@@ -204,8 +187,6 @@ class JoinTestBase
protected <T extends Geometry> void sanityCheckFlatJoinResults(List<Tuple2<Polygon, T>> results)
{
for (final Tuple2<Polygon, T> tuple : results) {
- assertNotNull(tuple._1().getUserData());
- assertNotNull(tuple._2().getUserData());
assertTrue(tuple._1().intersects(tuple._2()));
}
}
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/LineStringJoinTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/LineStringJoinTest.java
index 4600255..46ff4a6 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/LineStringJoinTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/LineStringJoinTest.java
@@ -34,7 +34,6 @@ import scala.Tuple2;
import java.util.Arrays;
import java.util.Collection;
-import java.util.HashSet;
import java.util.List;
import static org.junit.Assert.assertEquals;
@@ -47,20 +46,17 @@ public class LineStringJoinTest
private static long expectedMatchCount;
private static long expectedMatchWithOriginalDuplicatesCount;
- public LineStringJoinTest(GridType gridType, boolean useLegacyPartitionAPIs, int numPartitions)
+ public LineStringJoinTest(GridType gridType, int numPartitions)
{
- super(gridType, useLegacyPartitionAPIs, numPartitions);
+ super(gridType, numPartitions);
}
@Parameterized.Parameters
public static Collection testParams()
{
return Arrays.asList(new Object[][] {
- {GridType.RTREE, true, 11},
- {GridType.RTREE, false, 11},
- {GridType.QUADTREE, true, 11},
- {GridType.QUADTREE, false, 11},
- {GridType.KDBTREE, false, 11},
+ {GridType.QUADTREE, 11},
+ {GridType.KDBTREE, 11},
});
}
@@ -100,10 +96,12 @@ public class LineStringJoinTest
partitionRdds(queryRDD, spatialRDD);
- List<Tuple2<Polygon, HashSet<LineString>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<LineString>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
- assertEquals(expectedMatchCount, countJoinResults(result));
+ long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedMatchWithOriginalDuplicatesCount : expectedMatchCount;
+ assertEquals(expectedCount, countJoinResults(result));
}
/**
@@ -140,10 +138,12 @@ public class LineStringJoinTest
partitionRdds(queryRDD, spatialRDD);
spatialRDD.buildIndex(indexType, true);
- List<Tuple2<Polygon, HashSet<LineString>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<LineString>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
- assertEquals(expectedMatchCount, countJoinResults(result));
+ long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedMatchWithOriginalDuplicatesCount : expectedMatchCount;
+ assertEquals(expectedCount, countJoinResults(result));
}
@Test
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointJoinTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointJoinTest.java
index 4aeea03..a4aff87 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointJoinTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointJoinTest.java
@@ -36,7 +36,6 @@ import scala.Tuple2;
import java.util.Arrays;
import java.util.Collection;
-import java.util.HashSet;
import java.util.List;
import static org.junit.Assert.assertEquals;
@@ -51,20 +50,17 @@ public class PointJoinTest
private static long expectedPolygonMatchCount;
private static long expectedPolygonMatchWithOriginalDuplicatesCount;
- public PointJoinTest(GridType gridType, boolean useLegacyPartitionAPIs, int numPartitions)
+ public PointJoinTest(GridType gridType, int numPartitions)
{
- super(gridType, useLegacyPartitionAPIs, numPartitions);
+ super(gridType, numPartitions);
}
@Parameterized.Parameters
public static Collection testParams()
{
return Arrays.asList(new Object[][] {
- {GridType.RTREE, true, 11},
- {GridType.RTREE, false, 11},
- {GridType.QUADTREE, true, 11},
- {GridType.QUADTREE, false, 11},
- {GridType.KDBTREE, false, 11},
+ {GridType.QUADTREE, 11},
+ {GridType.KDBTREE, 11},
});
}
@@ -115,7 +111,9 @@ public class PointJoinTest
throws Exception
{
PolygonRDD queryRDD = createPolygonRDD();
- testNestedLoopInt(queryRDD, expectedPolygonMatchCount);
+ final long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedPolygonMatchWithOriginalDuplicatesCount : expectedPolygonMatchCount;
+ testNestedLoopInt(queryRDD, expectedCount);
}
private void testNestedLoopInt(SpatialRDD<Polygon> queryRDD, long expectedCount)
@@ -125,7 +123,7 @@ public class PointJoinTest
partitionRdds(queryRDD, spatialRDD);
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
assertEquals(expectedCount, countJoinResults(result));
@@ -154,7 +152,9 @@ public class PointJoinTest
throws Exception
{
PolygonRDD queryRDD = createPolygonRDD();
- testIndexInt(queryRDD, IndexType.RTREE, expectedPolygonMatchCount);
+ final long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedPolygonMatchWithOriginalDuplicatesCount : expectedPolygonMatchCount;
+ testIndexInt(queryRDD, IndexType.RTREE, expectedCount);
}
/**
@@ -180,7 +180,9 @@ public class PointJoinTest
throws Exception
{
PolygonRDD queryRDD = createPolygonRDD();
- testIndexInt(queryRDD, IndexType.QUADTREE, expectedPolygonMatchCount);
+ final long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedPolygonMatchWithOriginalDuplicatesCount : expectedPolygonMatchCount;
+ testIndexInt(queryRDD, IndexType.QUADTREE, expectedCount);
}
private void testIndexInt(SpatialRDD<Polygon> queryRDD, IndexType indexType, long expectedCount)
@@ -191,7 +193,7 @@ public class PointJoinTest
partitionRdds(queryRDD, spatialRDD);
spatialRDD.buildIndex(indexType, true);
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
assertEquals(expectedCount, countJoinResults(result));
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointKnnTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointKnnTest.java
index f7d0be3..aebe610 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointKnnTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointKnnTest.java
@@ -176,7 +176,6 @@ public class PointKnnTest
for (int i = 0; i < loopTimes; i++) {
List<Point> result = KNNQuery.SpatialKnnQuery(pointRDD, queryPoint, topK, false);
assert result.size() > -1;
- assert result.get(0).getUserData().toString() != null;
//System.out.println(result.get(0).getUserData().toString());
}
}
@@ -195,7 +194,6 @@ public class PointKnnTest
for (int i = 0; i < loopTimes; i++) {
List<Point> result = KNNQuery.SpatialKnnQuery(pointRDD, queryPoint, topK, true);
assert result.size() > -1;
- assert result.get(0).getUserData().toString() != null;
//System.out.println(result.get(0).getUserData().toString());
}
}
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointRangeTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointRangeTest.java
index 22cd54d..78d4eb6 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/PointRangeTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/PointRangeTest.java
@@ -166,7 +166,6 @@ public class PointRangeTest
long resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, false).count();
assertEquals(resultSize, 2830);
}
- assert RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, false).take(10).get(1).getUserData().toString() != null;
}
/**
@@ -184,6 +183,5 @@ public class PointRangeTest
long resultSize = RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, true).count();
assertEquals(resultSize, 2830);
}
- assert RangeQuery.SpatialRangeQuery(spatialRDD, queryEnvelope, false, true).take(10).get(1).getUserData().toString() != null;
}
}
\ No newline at end of file
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/PolygonJoinTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/PolygonJoinTest.java
index 22c0991..ece60f6 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/PolygonJoinTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/PolygonJoinTest.java
@@ -32,7 +32,6 @@ import scala.Tuple2;
import java.util.Arrays;
import java.util.Collection;
-import java.util.HashSet;
import java.util.List;
import static org.junit.Assert.assertEquals;
@@ -47,20 +46,17 @@ public class PolygonJoinTest
private static long expectedContainsWithOriginalDuplicatesCount;
private static long expectedIntersectsWithOriginalDuplicatesCount;
- public PolygonJoinTest(GridType gridType, boolean useLegacyPartitionAPIs, int numPartitions)
+ public PolygonJoinTest(GridType gridType, int numPartitions)
{
- super(gridType, useLegacyPartitionAPIs, numPartitions);
+ super(gridType, numPartitions);
}
@Parameterized.Parameters
public static Collection testParams()
{
return Arrays.asList(new Object[][] {
- {GridType.RTREE, true, 11},
- {GridType.RTREE, false, 11},
- {GridType.QUADTREE, true, 11},
- {GridType.QUADTREE, false, 11},
- {GridType.KDBTREE, false, 11},
+ {GridType.QUADTREE, 11},
+ {GridType.KDBTREE, 11},
});
}
@@ -160,9 +156,9 @@ public class PolygonJoinTest
partitionRdds(queryRDD, spatialRDD);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, intersects).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, intersects).collect();
sanityCheckJoinResults(result);
- assertEquals(getExpectedCount(intersects), countJoinResults(result));
+ assertEquals(getExpectedWithOriginalDuplicatesCount(intersects), countJoinResults(result));
}
/**
@@ -207,9 +203,9 @@ public class PolygonJoinTest
partitionRdds(queryRDD, spatialRDD);
spatialRDD.buildIndex(indexType, true);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, true, intersects).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, true, intersects).collect();
sanityCheckJoinResults(result);
- assertEquals(getExpectedCount(intersects), countJoinResults(result));
+ assertEquals(getExpectedWithOriginalDuplicatesCount(intersects), countJoinResults(result));
}
private long getExpectedCount(boolean intersects)
diff --git a/core/src/test/java/org/apache/sedona/core/spatialOperator/RectangleJoinTest.java b/core/src/test/java/org/apache/sedona/core/spatialOperator/RectangleJoinTest.java
index f08790f..87ffc2c 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialOperator/RectangleJoinTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialOperator/RectangleJoinTest.java
@@ -32,7 +32,6 @@ import scala.Tuple2;
import java.util.Arrays;
import java.util.Collection;
-import java.util.HashSet;
import java.util.List;
import static org.junit.Assert.assertEquals;
@@ -45,20 +44,17 @@ public class RectangleJoinTest
private static long expectedMatchCount;
private static long expectedMatchWithOriginalDuplicatesCount;
- public RectangleJoinTest(GridType gridType, boolean useLegacyPartitionAPIs, int numPartitions)
+ public RectangleJoinTest(GridType gridType, int numPartitions)
{
- super(gridType, useLegacyPartitionAPIs, numPartitions);
+ super(gridType, numPartitions);
}
@Parameterized.Parameters
public static Collection testParams()
{
return Arrays.asList(new Object[][] {
- {GridType.RTREE, true, 11},
- {GridType.RTREE, false, 11},
- {GridType.QUADTREE, true, 11},
- {GridType.QUADTREE, false, 11},
- {GridType.KDBTREE, false, 11},
+ {GridType.QUADTREE, 11},
+ {GridType.KDBTREE, 11},
});
}
@@ -97,10 +93,12 @@ public class RectangleJoinTest
partitionRdds(queryRDD, spatialRDD);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
- assertEquals(expectedMatchCount, countJoinResults(result));
+ long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedMatchWithOriginalDuplicatesCount : expectedMatchCount;
+ assertEquals(expectedCount, countJoinResults(result));
}
/**
@@ -136,10 +134,12 @@ public class RectangleJoinTest
partitionRdds(queryRDD, spatialRDD);
spatialRDD.buildIndex(indexType, true);
- List<Tuple2<Polygon, HashSet<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Polygon>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
sanityCheckJoinResults(result);
- assertEquals(expectedMatchCount, countJoinResults(result));
+ long expectedCount = expectToPreserveOriginalDuplicates()
+ ? expectedMatchWithOriginalDuplicatesCount : expectedMatchCount;
+ assertEquals(expectedCount, countJoinResults(result));
}
@Test
diff --git a/core/src/test/java/org/apache/sedona/core/spatialRDD/LineStringRDDTest.java b/core/src/test/java/org/apache/sedona/core/spatialRDD/LineStringRDDTest.java
index 5b6b159..fe7692e 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialRDD/LineStringRDDTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialRDD/LineStringRDDTest.java
@@ -18,13 +18,11 @@
*/
package org.apache.sedona.core.spatialRDD;
-import org.apache.sedona.core.enums.GridType;
import org.apache.sedona.core.enums.IndexType;
import org.apache.spark.storage.StorageLevel;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.locationtech.jts.geom.Envelope;
import org.locationtech.jts.geom.Polygon;
import org.locationtech.jts.index.strtree.STRtree;
@@ -88,54 +86,6 @@ public class LineStringRDDTest
}
/**
- * Test hilbert curve spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testHilbertCurveSpatialPartitioing()
- throws Exception
- {
- LineStringRDD spatialRDD = new LineStringRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.HILBERT);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
- * Test R tree spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testRTreeSpatialPartitioing()
- throws Exception
- {
- LineStringRDD spatialRDD = new LineStringRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
- * Test voronoi spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testVoronoiSpatialPartitioing()
- throws Exception
- {
- LineStringRDD spatialRDD = new LineStringRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.VORONOI);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
* Test build index without set grid.
*
* @throws Exception the exception
diff --git a/core/src/test/java/org/apache/sedona/core/spatialRDD/PointRDDTest.java b/core/src/test/java/org/apache/sedona/core/spatialRDD/PointRDDTest.java
index a0098a0..bca0563 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialRDD/PointRDDTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialRDD/PointRDDTest.java
@@ -18,13 +18,11 @@
*/
package org.apache.sedona.core.spatialRDD;
-import org.apache.sedona.core.enums.GridType;
import org.apache.sedona.core.enums.IndexType;
import org.apache.spark.storage.StorageLevel;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.locationtech.jts.geom.Envelope;
import org.locationtech.jts.geom.Point;
import org.locationtech.jts.index.strtree.STRtree;
@@ -89,74 +87,6 @@ public class PointRDDTest
}
/**
- * Test equal partitioning.
- *
- * @throws Exception the exception
- */
- @Test
- public void testEqualPartitioning()
- throws Exception
- {
- PointRDD spatialRDD = new PointRDD(sc, InputLocation, offset, splitter, false, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.EQUALGRID);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
- * Test hilbert curve spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testHilbertCurveSpatialPartitioing()
- throws Exception
- {
- PointRDD spatialRDD = new PointRDD(sc, InputLocation, offset, splitter, false, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.HILBERT);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
- * Test R tree spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testRTreeSpatialPartitioing()
- throws Exception
- {
- PointRDD spatialRDD = new PointRDD(sc, InputLocation, offset, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
- * Test voronoi spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testVoronoiSpatialPartitioing()
- throws Exception
- {
- PointRDD spatialRDD = new PointRDD(sc, InputLocation, offset, splitter, false, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.VORONOI);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
* Test build index without set grid.
*
* @throws Exception the exception
diff --git a/core/src/test/java/org/apache/sedona/core/spatialRDD/PolygonRDDTest.java b/core/src/test/java/org/apache/sedona/core/spatialRDD/PolygonRDDTest.java
index b8d0267..dfae6cc 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialRDD/PolygonRDDTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialRDD/PolygonRDDTest.java
@@ -19,13 +19,11 @@
package org.apache.sedona.core.spatialRDD;
import org.apache.sedona.core.enums.FileDataSplitter;
-import org.apache.sedona.core.enums.GridType;
import org.apache.sedona.core.enums.IndexType;
import org.apache.spark.storage.StorageLevel;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.locationtech.jts.geom.Envelope;
import org.locationtech.jts.geom.Polygon;
import org.locationtech.jts.index.strtree.STRtree;
@@ -126,54 +124,6 @@ public class PolygonRDDTest
}
/**
- * Test hilbert curve spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testHilbertCurveSpatialPartitioing()
- throws Exception
- {
- PolygonRDD spatialRDD = new PolygonRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.HILBERT);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
- * Test R tree spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testRTreeSpatialPartitioing()
- throws Exception
- {
- PolygonRDD spatialRDD = new PolygonRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
- * Test voronoi spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testVoronoiSpatialPartitioing()
- throws Exception
- {
- PolygonRDD spatialRDD = new PolygonRDD(sc, InputLocation, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.VORONOI);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- }
-
- /**
* Test build index without set grid.
*
* @throws Exception the exception
diff --git a/core/src/test/java/org/apache/sedona/core/spatialRDD/RectangleRDDTest.java b/core/src/test/java/org/apache/sedona/core/spatialRDD/RectangleRDDTest.java
index 03b6682..aa63f21 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialRDD/RectangleRDDTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialRDD/RectangleRDDTest.java
@@ -18,13 +18,11 @@
*/
package org.apache.sedona.core.spatialRDD;
-import org.apache.sedona.core.enums.GridType;
import org.apache.sedona.core.enums.IndexType;
import org.apache.spark.storage.StorageLevel;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
-import org.locationtech.jts.geom.Envelope;
import org.locationtech.jts.geom.Point;
import org.locationtech.jts.index.strtree.STRtree;
@@ -87,57 +85,6 @@ public class RectangleRDDTest
}
/**
- * Test hilbert curve spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testHilbertCurveSpatialPartitioing()
- throws Exception
- {
- RectangleRDD spatialRDD = new RectangleRDD(sc, InputLocation, offset, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.HILBERT);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
- * Test R tree spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testRTreeSpatialPartitioing()
- throws Exception
- {
- RectangleRDD spatialRDD = new RectangleRDD(sc, InputLocation, offset, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
- * Test voronoi spatial partitioing.
- *
- * @throws Exception the exception
- */
- @Test
- public void testVoronoiSpatialPartitioing()
- throws Exception
- {
- RectangleRDD spatialRDD = new RectangleRDD(sc, InputLocation, offset, splitter, true, 10, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.VORONOI);
- for (Envelope d : spatialRDD.grids) {
- //System.out.println("PointRDD spatial partitioning grids: "+d.grid);
- }
- assert spatialRDD.countWithoutDuplicates() == spatialRDD.countWithoutDuplicatesSPRDD();
- }
-
- /**
* Test build index without set grid.
*
* @throws Exception the exception
diff --git a/core/src/test/java/org/apache/sedona/core/spatialRDD/SpatialRDDWriterTest.java b/core/src/test/java/org/apache/sedona/core/spatialRDD/SpatialRDDWriterTest.java
index 1ae7685..8931f0c 100644
--- a/core/src/test/java/org/apache/sedona/core/spatialRDD/SpatialRDDWriterTest.java
+++ b/core/src/test/java/org/apache/sedona/core/spatialRDD/SpatialRDDWriterTest.java
@@ -22,15 +22,19 @@ package org.apache.sedona.core.spatialRDD;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang.NullArgumentException;
import org.apache.sedona.core.enums.FileDataSplitter;
+import org.apache.sedona.core.utils.GeomUtils;
import org.apache.spark.storage.StorageLevel;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
+import org.locationtech.jts.geom.Point;
import java.io.File;
import java.io.IOException;
+import java.util.List;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
public class SpatialRDDWriterTest
extends SpatialRDDTestBase
@@ -85,7 +89,7 @@ public class SpatialRDDWriterTest
PointRDD resultWKB = new PointRDD(sc, testSaveAsWKBWithData, 0, FileDataSplitter.WKB, true, numPartitions, StorageLevel.MEMORY_ONLY());
assertEquals(resultWKB.rawSpatialRDD.count(), spatialRDD.rawSpatialRDD.count());
- assertEquals(resultWKB.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
+ verifyResult(resultWKB.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
}
/**
@@ -105,7 +109,7 @@ public class SpatialRDDWriterTest
PointRDD resultWKT = new PointRDD(sc, testSaveAsWKTWithData, 0, FileDataSplitter.WKT, true, numPartitions, StorageLevel.MEMORY_ONLY());
assertEquals(resultWKT.rawSpatialRDD.count(), spatialRDD.rawSpatialRDD.count());
- assertEquals(resultWKT.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
+ verifyResult(resultWKT.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
}
/**
@@ -125,7 +129,7 @@ public class SpatialRDDWriterTest
PointRDD resultWKB = new PointRDD(sc, testSaveAsWKB, 0, FileDataSplitter.WKB, false, numPartitions, StorageLevel.MEMORY_ONLY());
assertEquals(resultWKB.rawSpatialRDD.count(), spatialRDD.rawSpatialRDD.count());
- assertEquals(resultWKB.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
+ verifyResult(resultWKB.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
}
/**
@@ -145,7 +149,7 @@ public class SpatialRDDWriterTest
PointRDD resultWKT = new PointRDD(sc, testSaveAsWKT, 0, FileDataSplitter.WKT, false, numPartitions, StorageLevel.MEMORY_ONLY());
assertEquals(resultWKT.rawSpatialRDD.count(), spatialRDD.rawSpatialRDD.count());
- assertEquals(resultWKT.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
+ verifyResult(resultWKT.rawSpatialRDD.takeOrdered(5), spatialRDD.rawSpatialRDD.takeOrdered(5));
}
/**
@@ -158,4 +162,11 @@ public class SpatialRDDWriterTest
PointRDD emptySpatialRDD = new PointRDD();
emptySpatialRDD.saveAsWKB(testSaveAsEmptyWKB);
}
+
+ private void verifyResult(List<Point> left, List<Point> right){
+ assertEquals(left.size(), right.size());
+ for (int i = 0; i < left.size(); i++) {
+ assertTrue(GeomUtils.equalsExactGeom(left.get(i), right.get(i)));
+ }
+ }
}
diff --git a/core/src/test/java/org/apache/sedona/core/utils/CRSTransformationTest.java b/core/src/test/java/org/apache/sedona/core/utils/CRSTransformationTest.java
index 4f8ff81..6a5fd79 100644
--- a/core/src/test/java/org/apache/sedona/core/utils/CRSTransformationTest.java
+++ b/core/src/test/java/org/apache/sedona/core/utils/CRSTransformationTest.java
@@ -49,7 +49,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Collections;
-import java.util.HashSet;
import java.util.List;
import java.util.Properties;
@@ -316,9 +315,9 @@ public class CRSTransformationTest
spatialRDD.spatialPartitioning(gridType);
- queryRDD.spatialPartitioning(spatialRDD.grids);
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
assert result.get(1)._1().getUserData() != null;
for (int i = 0; i < result.size(); i++) {
@@ -344,9 +343,9 @@ public class CRSTransformationTest
spatialRDD.buildIndex(IndexType.RTREE, true);
- queryRDD.spatialPartitioning(spatialRDD.grids);
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
- List<Tuple2<Polygon, HashSet<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
+ List<Tuple2<Polygon, List<Point>>> result = JoinQuery.SpatialJoinQuery(spatialRDD, queryRDD, false, true).collect();
assert result.get(1)._1().getUserData() != null;
for (int i = 0; i < result.size(); i++) {
@@ -367,14 +366,14 @@ public class CRSTransformationTest
CircleRDD windowRDD = new CircleRDD(queryRDD, 0.1);
PolygonRDD objectRDD = new PolygonRDD(sc, InputLocationQueryPolygon, splitter, true, numPartitions, StorageLevel.MEMORY_ONLY(), "epsg:4326", "epsg:3857");
objectRDD.rawSpatialRDD.repartition(4);
- objectRDD.spatialPartitioning(GridType.RTREE);
+ objectRDD.spatialPartitioning(GridType.KDBTREE);
objectRDD.buildIndex(IndexType.RTREE, true);
- windowRDD.spatialPartitioning(objectRDD.grids);
+ windowRDD.spatialPartitioning(objectRDD.getPartitioner());
- List<Tuple2<Geometry, HashSet<Polygon>>> results = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, false).collect();
+ List<Tuple2<Geometry, List<Polygon>>> results = JoinQuery.DistanceJoinQuery(objectRDD, windowRDD, true, false).collect();
assertEquals(5467, results.size());
- for (Tuple2<Geometry, HashSet<Polygon>> tuple : results) {
+ for (Tuple2<Geometry, List<Polygon>> tuple : results) {
for (Polygon polygon : tuple._2()) {
assertTrue(new Circle(tuple._1(), 0.1).covers(polygon));
}
diff --git a/core/src/test/resources/babylon.linestring.properties b/core/src/test/resources/babylon.linestring.properties
deleted file mode 100644
index f9b7b71..0000000
--- a/core/src/test/resources/babylon.linestring.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=primaryroads-linestring.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/core/src/test/resources/babylon.point.properties b/core/src/test/resources/babylon.point.properties
deleted file mode 100644
index 3fb21a6..0000000
--- a/core/src/test/resources/babylon.point.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=arealm.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/core/src/test/resources/babylon.polygon.properties b/core/src/test/resources/babylon.polygon.properties
deleted file mode 100644
index ef76396..0000000
--- a/core/src/test/resources/babylon.polygon.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=primaryroads-polygon.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/core/src/test/resources/babylon.rectangle.properties b/core/src/test/resources/babylon.rectangle.properties
deleted file mode 100644
index 54a283d..0000000
--- a/core/src/test/resources/babylon.rectangle.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=zcta510.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/core/src/test/resources/crs.test.properties b/core/src/test/resources/crs.test.properties
index eebfcd3..a010c5c 100644
--- a/core/src/test/resources/crs.test.properties
+++ b/core/src/test/resources/crs.test.properties
@@ -1,7 +1,7 @@
inputLocation=crs-test-point.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.01
diff --git a/core/src/test/resources/linestring.test.properties b/core/src/test/resources/linestring.test.properties
index fdfe140..6b1d5ec 100644
--- a/core/src/test/resources/linestring.test.properties
+++ b/core/src/test/resources/linestring.test.properties
@@ -2,7 +2,7 @@ inputLocation=primaryroads-linestring.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=5
distance=0.01
diff --git a/core/src/test/resources/point.test.properties b/core/src/test/resources/point.test.properties
index 2479a87..2a308a4 100644
--- a/core/src/test/resources/point.test.properties
+++ b/core/src/test/resources/point.test.properties
@@ -2,7 +2,7 @@ inputLocation=arealm-small.csv
queryWindowSet=zcta510-small.csv
offset=1
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.01
diff --git a/core/src/test/resources/polygon.test.properties b/core/src/test/resources/polygon.test.properties
index 960f49f..05526e1 100644
--- a/core/src/test/resources/polygon.test.properties
+++ b/core/src/test/resources/polygon.test.properties
@@ -2,7 +2,7 @@ inputLocation=primaryroads-polygon.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=5
distance=0.01
diff --git a/core/src/test/resources/rectangle.test.properties b/core/src/test/resources/rectangle.test.properties
index f61871b..26f3a24 100644
--- a/core/src/test/resources/rectangle.test.properties
+++ b/core/src/test/resources/rectangle.test.properties
@@ -2,7 +2,7 @@ inputLocation=zcta510-small.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.001
diff --git a/jts b/jts
index 41df452..e4d7d6f 160000
--- a/jts
+++ b/jts
@@ -1 +1 @@
-Subproject commit 41df452efc4ce7c76499e35a0bf4526d771165f9
+Subproject commit e4d7d6f451c908b1bfeef6a10230a5220d7e28d1
diff --git a/pom.xml b/pom.xml
index 898f877..77cd6d0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -40,7 +40,7 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<scala.version>2.12.8</scala.version>
<scala.compat.version>2.12</scala.compat.version>
- <geotools.version>23.2</geotools.version>
+ <geotools.version>24.0</geotools.version>
<spark.version>3.0.0</spark.version>
</properties>
<dependencies>
diff --git a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/adapters/PythonConverter.scala b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/adapters/PythonConverter.scala
index 2e1e9d6..8065a55 100644
--- a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/adapters/PythonConverter.scala
+++ b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/adapters/PythonConverter.scala
@@ -19,7 +19,7 @@
package org.apache.sedona.python.wrapper.adapters
-import org.apache.sedona.python.wrapper.translation.{FlatPairRddConverter, GeometryRddConverter, GeometrySeqToPythonConverter, HashSetPairRddConverter, PythonRDDToJavaConverter}
+import org.apache.sedona.python.wrapper.translation.{FlatPairRddConverter, GeometryRddConverter, GeometrySeqToPythonConverter, ListPairRddConverter, PythonRDDToJavaConverter}
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
import org.locationtech.jts.geom.Geometry
@@ -33,8 +33,8 @@ object PythonConverter extends GeomSerializer {
def translateSpatialPairRDDToPython(spatialRDD: JavaPairRDD[Geometry, Geometry]): JavaRDD[Array[Byte]] =
FlatPairRddConverter(spatialRDD, geometrySerializer).translateToPython
- def translateSpatialPairRDDWithHashSetToPython(spatialRDD: JavaPairRDD[Geometry, java.util.HashSet[Geometry]]): JavaRDD[Array[Byte]] =
- HashSetPairRddConverter(spatialRDD, geometrySerializer).translateToPython
+ def translateSpatialPairRDDWithListToPython(spatialRDD: JavaPairRDD[Geometry, java.util.List[Geometry]]): JavaRDD[Array[Byte]] =
+ ListPairRddConverter(spatialRDD, geometrySerializer).translateToPython
def translatePythonRDDToJava(pythonRDD: JavaRDD[Array[Byte]]): JavaRDD[Geometry] =
PythonRDDToJavaConverter(pythonRDD, geometrySerializer).translateToJava
diff --git a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/HashSetPairRddConverter.scala b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/ListPairRddConverter.scala
similarity index 86%
rename from python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/HashSetPairRddConverter.scala
rename to python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/ListPairRddConverter.scala
index 27997df..993d3bc 100644
--- a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/HashSetPairRddConverter.scala
+++ b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/translation/ListPairRddConverter.scala
@@ -23,8 +23,8 @@ import org.apache.sedona.python.wrapper.utils.implicits._
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
import org.locationtech.jts.geom.Geometry
-private[python] case class HashSetPairRddConverter(spatialRDD: JavaPairRDD[Geometry, java.util.HashSet[Geometry]],
- geometrySerializer: PythonGeometrySerializer) extends RDDToPythonConverter {
+private[python] case class ListPairRddConverter(spatialRDD: JavaPairRDD[Geometry, java.util.List[Geometry]],
+ geometrySerializer: PythonGeometrySerializer) extends RDDToPythonConverter {
override def translateToPython: JavaRDD[Array[Byte]] = {
spatialRDD.rdd.map[Array[Byte]](
pairRDD => {
diff --git a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/utils/implicits.scala b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/utils/implicits.scala
index 951a034..36d62a6 100644
--- a/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/utils/implicits.scala
+++ b/python-adapter/src/main/scala/org.apache.sedona.python.wrapper/utils/implicits.scala
@@ -21,7 +21,6 @@ package org.apache.sedona.python.wrapper.utils
import java.nio.charset.StandardCharsets
import java.nio.{ByteBuffer, ByteOrder}
-import java.util
import org.locationtech.jts.geom.Geometry
@@ -48,16 +47,13 @@ object implicits {
}
implicit class GeometryEnhancer(geometry: Geometry) {
- def userDataToUtf8ByteArray: Array[Byte] =
- geometry.getUserData.asInstanceOf[String]
- .getBytes(StandardCharsets.UTF_8)
- }
+ private val EMPTY_STRING = ""
- implicit class ListConverter[T](elements: List[T]) {
- def toJavaHashSet: java.util.HashSet[T] = {
- val javaHashSet = new util.HashSet[T]()
- elements.foreach(javaHashSet.add)
- javaHashSet
+ def userDataToUtf8ByteArray: Array[Byte] = {
+ geometry.getUserData match {
+ case null => EMPTY_STRING.getBytes(StandardCharsets.UTF_8)
+ case data: String => data.getBytes(StandardCharsets.UTF_8)
+ }
}
}
diff --git a/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/GeometrySample.scala b/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/GeometrySample.scala
index 8e9b4e6..686c00c 100644
--- a/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/GeometrySample.scala
+++ b/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/GeometrySample.scala
@@ -3,6 +3,7 @@ package org.apache.sedona.python.wrapper
import org.locationtech.jts.geom.Geometry
import scala.tools.nsc.interpreter.InputStream
+import scala.io.Source
trait GeometrySample extends PythonTestSpec {
self: TestToPythonSerialization =>
@@ -29,7 +30,6 @@ trait GeometrySample extends PythonTestSpec {
private def loadResourceFile(fileName: String): List[String] = {
val stream: InputStream = getClass.getResourceAsStream(fileName)
- val lines: Iterator[String] = scala.io.Source.fromInputStream(stream).getLines
- lines.toList
+ Source.fromInputStream(stream).getLines.toList
}
}
diff --git a/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/TestToPythonSerialization.scala b/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/TestToPythonSerialization.scala
index 21fd168..96503c8 100644
--- a/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/TestToPythonSerialization.scala
+++ b/python-adapter/src/test/scala/org/apache/sedona/python/wrapper/TestToPythonSerialization.scala
@@ -19,11 +19,12 @@
package org.apache.sedona.python.wrapper
-import org.apache.sedona.python.wrapper.translation.{FlatPairRddConverter, GeometryRddConverter, HashSetPairRddConverter}
-import org.apache.sedona.python.wrapper.utils.implicits.ListConverter
+import org.apache.sedona.python.wrapper.translation.{FlatPairRddConverter, GeometryRddConverter, ListPairRddConverter}
import org.apache.spark.api.java.JavaPairRDD
import org.scalatest.Matchers
import org.apache.sedona.python.wrapper.utils.implicits._
+import scala.collection.JavaConverters._
+
class TestToPythonSerialization extends SparkUtil with GeometrySample with Matchers {
@@ -45,13 +46,13 @@ class TestToPythonSerialization extends SparkUtil with GeometrySample with Match
}) should contain theSameElementsAs expectedPairRDDPythonArray
}
- test("Should serialize to Python JavaRDD[Geometry, HashSet[Geometry]]") {
- val translatedToPythonHashSet = HashSetPairRddConverter(
- JavaPairRDD.fromRDD(spatialPairRDDWithHashSet), pythonGeometrySerializer).translateToPython
- val existingValues = translatedToPythonHashSet.collect().toArray().toList.map {
+ test("Should serialize to Python JavaRDD[Geometry, List[Geometry]]") {
+ val translatedToPythonList = ListPairRddConverter(
+ JavaPairRDD.fromRDD(spatialPairRDDWithList), pythonGeometrySerializer).translateToPython
+ val existingValues = translatedToPythonList.collect().toArray().toList.map {
case a: Array[Byte] => a.toList
}
- existingValues should contain theSameElementsAs expectedPairRDDWithHashSetPythonArray
+ existingValues should contain theSameElementsAs expectedPairRDDWithListPythonArray
}
private val pointSpatialRDD = sc.parallelize(
@@ -64,9 +65,9 @@ class TestToPythonSerialization extends SparkUtil with GeometrySample with Match
)
)
- private val spatialPairRDDWithHashSet = sc.parallelize(
+ private val spatialPairRDDWithList = sc.parallelize(
samplePolygons.map(
- polygon => (polygon, samplePoints.slice(0, 2).toJavaHashSet)
+ polygon => (polygon, samplePoints.slice(0, 2).asJava)
)
)
@@ -79,7 +80,7 @@ class TestToPythonSerialization extends SparkUtil with GeometrySample with Match
++ 1.toByteArray().toList ++ pythonGeometrySerializer.serialize(geometries._2).toList
)
- private val expectedPairRDDWithHashSetPythonArray: List[List[Byte]] = samplePolygons.map(
+ private val expectedPairRDDWithListPythonArray: List[List[Byte]] = samplePolygons.map(
samplePolygon => 1.toByteArray().toList ++ pythonGeometrySerializer.serialize(samplePolygon).toList ++ 2.toByteArray() ++
samplePoints.slice(0, 2).flatMap(samplePoint => pythonGeometrySerializer.serialize(samplePoint)))
diff --git a/python/sedona/core/SpatialRDD/__init__.py b/python/sedona/core/SpatialRDD/__init__.py
index dffa127..e51e547 100644
--- a/python/sedona/core/SpatialRDD/__init__.py
+++ b/python/sedona/core/SpatialRDD/__init__.py
@@ -25,4 +25,4 @@ from .rectangle_rdd import SpatialRDD
__all__ = [
"PolygonRDD", "PointRDD", "CircleRDD", "LineStringRDD", "RectangleRDD", "SpatialRDD"
-]
\ No newline at end of file
+]
diff --git a/python/sedona/core/SpatialRDD/linestring_rdd.py b/python/sedona/core/SpatialRDD/linestring_rdd.py
index b3a584b..dff4e78 100644
--- a/python/sedona/core/SpatialRDD/linestring_rdd.py
+++ b/python/sedona/core/SpatialRDD/linestring_rdd.py
@@ -71,7 +71,7 @@ class LineStringRDD(SpatialRDD, metaclass=MultipleMeta):
self._srdd = self._jvm_spatial_rdd(jsrdd, sourceEpsgCode, targetEpsgCode)
def __init__(self, sparkContext: SparkContext, InputLocation: str, startOffset: int, endOffset: int,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int):
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int):
"""
:param sparkContext: SparkContext instance
@@ -139,7 +139,8 @@ class LineStringRDD(SpatialRDD, metaclass=MultipleMeta):
partitions
)
- def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool):
+ def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter,
+ carryInputData: bool):
"""
:param sparkContext: SparkContext instance
@@ -243,7 +244,7 @@ class LineStringRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool,
- newLevel: StorageLevel):
+ newLevel: StorageLevel):
"""
:param sparkContext: SparkContext instance
diff --git a/python/sedona/core/SpatialRDD/point_rdd.py b/python/sedona/core/SpatialRDD/point_rdd.py
index 7004d42..6354742 100644
--- a/python/sedona/core/SpatialRDD/point_rdd.py
+++ b/python/sedona/core/SpatialRDD/point_rdd.py
@@ -78,7 +78,6 @@ class PointRDD(SpatialRDD, metaclass=MultipleMeta):
jsrdd = rawSpatialRDD.jsrdd
self._srdd = self._jvm_spatial_rdd(jsrdd, sourceEpsgCode, targetEpsgCode)
-
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int, splitter: FileDataSplitter,
carryInputData: bool, partitions: int):
"""
@@ -142,7 +141,8 @@ class PointRDD(SpatialRDD, metaclass=MultipleMeta):
partitions
)
- def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool):
+ def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter,
+ carryInputData: bool):
"""
:param sparkContext: SparkContext instance
@@ -268,7 +268,8 @@ class PointRDD(SpatialRDD, metaclass=MultipleMeta):
new_level_jvm
)
- def __init__(self, rawSpatialRDD: JvmSpatialRDD, newLevel: StorageLevel, sourceEpsgCRSCode: str, targetEpsgCode: str):
+ def __init__(self, rawSpatialRDD: JvmSpatialRDD, newLevel: StorageLevel, sourceEpsgCRSCode: str,
+ targetEpsgCode: str):
"""
:param rawSpatialRDD: JvmSpatialRDD, jvm representation of spatial rdd
@@ -285,7 +286,6 @@ class PointRDD(SpatialRDD, metaclass=MultipleMeta):
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int, splitter: FileDataSplitter,
carryInputData: bool, partitions: int, newLevel: StorageLevel, sourceEpsgCRSCode: str,
targetEpsgCode: str):
-
"""
:param sparkContext: SparkContext instance
diff --git a/python/sedona/core/SpatialRDD/polygon_rdd.py b/python/sedona/core/SpatialRDD/polygon_rdd.py
index 7e8f2b0..a2758eb 100644
--- a/python/sedona/core/SpatialRDD/polygon_rdd.py
+++ b/python/sedona/core/SpatialRDD/polygon_rdd.py
@@ -149,7 +149,8 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
partitions
)
- def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool):
+ def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter,
+ carryInputData: bool):
"""
:param sparkContext: SparkContext, the spark context
@@ -169,7 +170,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, startOffset: int, endOffset: int,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -197,7 +198,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, startOffset: int, endOffset: int,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -224,7 +225,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -249,7 +250,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -286,7 +287,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
self._srdd = self._jvm_spatial_rdd(jsrdd, new_level_jvm, sourceEpsgCRSCode, targetEpsgCode)
def __init__(self, sparkContext: SparkContext, InputLocation: str, startOffset: int, endOffset: int,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel,
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel,
sourceEpsgCRSCode: str, targetEpsgCode: str):
"""
@@ -320,8 +321,8 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, startOffset: int, endOffset: int,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel, sourceEpsgCRSCode: str,
- targetEpsgCode: str):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel, sourceEpsgCRSCode: str,
+ targetEpsgCode: str):
"""
:param sparkContext: SparkContext, the spark context
@@ -351,7 +352,7 @@ class PolygonRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool,
- partitions: int, newLevel: StorageLevel, sourceEpsgCRSCode: str, targetEpsgCode: str):
+ partitions: int, newLevel: StorageLevel, sourceEpsgCRSCode: str, targetEpsgCode: str):
"""
:param sparkContext: SparkContext, the spark context
diff --git a/python/sedona/core/SpatialRDD/rectangle_rdd.py b/python/sedona/core/SpatialRDD/rectangle_rdd.py
index 06c3716..5851bb0 100644
--- a/python/sedona/core/SpatialRDD/rectangle_rdd.py
+++ b/python/sedona/core/SpatialRDD/rectangle_rdd.py
@@ -145,7 +145,8 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
partitions
)
- def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool):
+ def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter,
+ carryInputData: bool):
"""
:param sparkContext: SparkContext, the spark context
@@ -165,7 +166,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -191,7 +192,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -216,7 +217,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -241,7 +242,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel):
"""
:param sparkContext: SparkContext, the spark context
@@ -278,7 +279,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
self._srdd = self._jvm_spatial_rdd(jsrdd, new_level_jvm, sourceEpsgCRSCode, targetEpsgCode)
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int,
- splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel,
+ splitter: FileDataSplitter, carryInputData: bool, partitions: int, newLevel: StorageLevel,
sourceEpsgCRSCode: str, targetEpsgCode: str):
"""
@@ -310,8 +311,8 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, Offset: int,
- splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel, sourceEpsgCRSCode: str,
- targetEpsgCode: str):
+ splitter: FileDataSplitter, carryInputData: bool, newLevel: StorageLevel, sourceEpsgCRSCode: str,
+ targetEpsgCode: str):
"""
:param sparkContext: SparkContext, the spark context
@@ -339,7 +340,7 @@ class RectangleRDD(SpatialRDD, metaclass=MultipleMeta):
)
def __init__(self, sparkContext: SparkContext, InputLocation: str, splitter: FileDataSplitter, carryInputData: bool,
- partitions: int, newLevel: StorageLevel, sourceEpsgCRSCode: str, targetEpsgCode: str):
+ partitions: int, newLevel: StorageLevel, sourceEpsgCRSCode: str, targetEpsgCode: str):
"""
:param sparkContext: SparkContext, the spark context
diff --git a/python/sedona/core/SpatialRDD/spatial_rdd.py b/python/sedona/core/SpatialRDD/spatial_rdd.py
index ff99b3a..3a51c2c 100644
--- a/python/sedona/core/SpatialRDD/spatial_rdd.py
+++ b/python/sedona/core/SpatialRDD/spatial_rdd.py
@@ -30,7 +30,6 @@ from sedona.core.enums.index_type import IndexTypeJvm, IndexType
from sedona.core.enums.spatial import SpatialType
from sedona.core.geom.envelope import Envelope
from sedona.core.jvm.config import since
-from sedona.core.jvm.partitioner import JvmPartitioner
from sedona.core.jvm.translate import SedonaPythonConverter, JvmSedonaPythonConverter
from sedona.utils.decorators import require
from sedona.utils.jvm import JvmStorageLevel
@@ -224,7 +223,8 @@ class SpatialRDD:
:return:
"""
- serialized_spatial_rdd = SedonaPythonConverter(self._jvm).translate_spatial_rdd_to_python(self._srdd.getRawSpatialRDD())
+ serialized_spatial_rdd = SedonaPythonConverter(self._jvm).translate_spatial_rdd_to_python(
+ self._srdd.getRawSpatialRDD())
if not hasattr(self, "_raw_spatial_rdd"):
RDD.saveAsObjectFile = lambda x, path: x._jrdd.saveAsObjectFile(path)
@@ -322,15 +322,6 @@ class SpatialRDD:
self._indexed_raw_rdd = indexed_raw_rdd
@property
- def partitionTree(self) -> JvmPartitioner:
- """
-
- :return:
- """
-
- return JvmPartitioner(get_field(self._srdd, "partitionTree"))
-
- @property
def rawSpatialRDD(self):
"""
@@ -405,7 +396,7 @@ class SpatialRDD:
return getattr(self, "_spatial_partitioned_rdd")
- def spatialPartitioning(self, partitioning: Union[str, GridType, SpatialPartitioner, List[Envelope], JvmPartitioner],
+ def spatialPartitioning(self, partitioning: Union[str, GridType, SpatialPartitioner, List[Envelope]],
num_partitions: Optional[int] = None) -> bool:
"""
@@ -415,8 +406,6 @@ class SpatialRDD:
"""
if type(partitioning) == str:
grid = GridTypeJvm(self._jvm, GridType.from_str(partitioning)).jvm_instance
- elif type(partitioning) == JvmPartitioner:
- grid = partitioning.jpart
elif type(partitioning) == GridType:
grid = GridTypeJvm(self._jvm, partitioning).jvm_instance
elif type(partitioning) == SpatialPartitioner:
diff --git a/python/sedona/core/SpatialRDD/spatial_rdd_factory.py b/python/sedona/core/SpatialRDD/spatial_rdd_factory.py
index e778294..f44611e 100644
--- a/python/sedona/core/SpatialRDD/spatial_rdd_factory.py
+++ b/python/sedona/core/SpatialRDD/spatial_rdd_factory.py
@@ -25,7 +25,6 @@ from sedona.utils.decorators import require
@attr.s
class SpatialRDDFactory(ABC):
-
sparkContext = attr.ib(type=SparkContext)
def __attrs_post_init__(self):
diff --git a/python/sedona/core/enums/file_data_splitter.py b/python/sedona/core/enums/file_data_splitter.py
index 4323323..d0ecdbe 100644
--- a/python/sedona/core/enums/file_data_splitter.py
+++ b/python/sedona/core/enums/file_data_splitter.py
@@ -24,7 +24,6 @@ from sedona.utils.decorators import require
class FileDataSplitter(Enum):
-
CSV = "CSV"
TSV = "TSV"
GEOJSON = "GEOJSON"
@@ -45,7 +44,6 @@ class FileDataSplitter(Enum):
@attr.s
class FileSplitterJvm(JvmObject):
-
splitter = attr.ib(type=FileDataSplitter)
def _create_jvm_instance(self):
diff --git a/python/sedona/core/enums/grid_type.py b/python/sedona/core/enums/grid_type.py
index 3aa4cf8..b5d816b 100644
--- a/python/sedona/core/enums/grid_type.py
+++ b/python/sedona/core/enums/grid_type.py
@@ -24,11 +24,6 @@ from sedona.utils.decorators import require
class GridType(Enum):
-
- EQUALGRID = "EQUALGRID"
- HILBERT = "HILBERT"
- RTREE = "RTREE"
- VORONOI = "VORONOI"
QUADTREE = "QUADTREE"
KDBTREE = "KDBTREE"
@@ -43,7 +38,6 @@ class GridType(Enum):
@attr.s
class GridTypeJvm(JvmObject):
-
grid = attr.ib(type=GridType)
def _create_jvm_instance(self):
diff --git a/python/sedona/core/enums/index_type.py b/python/sedona/core/enums/index_type.py
index 6b655bb..290f43a 100644
--- a/python/sedona/core/enums/index_type.py
+++ b/python/sedona/core/enums/index_type.py
@@ -24,7 +24,6 @@ from sedona.utils.decorators import require
class IndexType(Enum):
-
QUADTREE = "QUADTREE"
RTREE = "RTREE"
@@ -39,7 +38,6 @@ class IndexType(Enum):
@attr.s
class IndexTypeJvm(JvmObject):
-
index_type = attr.ib(type=IndexType)
def _create_jvm_instance(self):
diff --git a/python/sedona/core/enums/join_build_side.py b/python/sedona/core/enums/join_build_side.py
index ca3eaf5..f79c159 100644
--- a/python/sedona/core/enums/join_build_side.py
+++ b/python/sedona/core/enums/join_build_side.py
@@ -26,4 +26,4 @@ class JoinBuildSide:
@classproperty
def RIGHT(self):
- return "RIGHT"
\ No newline at end of file
+ return "RIGHT"
diff --git a/python/sedona/core/formatMapper/disc_utils.py b/python/sedona/core/formatMapper/disc_utils.py
index ce5b09b..2fd3872 100644
--- a/python/sedona/core/formatMapper/disc_utils.py
+++ b/python/sedona/core/formatMapper/disc_utils.py
@@ -115,5 +115,4 @@ def load_spatial_rdd_from_disc(sc: SparkContext, path: str, geometry_type: GeoTy
def load_spatial_index_rdd_from_disc(sc: SparkContext, path: str):
-
- return IndexDiscLoader.load(sc, path)
\ No newline at end of file
+ return IndexDiscLoader.load(sc, path)
diff --git a/python/sedona/core/formatMapper/geo_json_reader.py b/python/sedona/core/formatMapper/geo_json_reader.py
index a1ba757..8296b2b 100644
--- a/python/sedona/core/formatMapper/geo_json_reader.py
+++ b/python/sedona/core/formatMapper/geo_json_reader.py
@@ -79,7 +79,8 @@ class GeoJsonReader(GeoDataReader, metaclass=MultipleMeta):
return spatial_rdd
@classmethod
- def readToGeometryRDD(cls, rawTextRDD: RDD, allowInvalidGeometries: bool, skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
+ def readToGeometryRDD(cls, rawTextRDD: RDD, allowInvalidGeometries: bool,
+ skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
"""
:param rawTextRDD: RDD
diff --git a/python/sedona/core/formatMapper/geo_reader.py b/python/sedona/core/formatMapper/geo_reader.py
index b6d0d26..d60b805 100644
--- a/python/sedona/core/formatMapper/geo_reader.py
+++ b/python/sedona/core/formatMapper/geo_reader.py
@@ -26,9 +26,6 @@ from sedona.utils.meta import MultipleMeta
class GeoDataReader(metaclass=MultipleMeta):
@abc.abstractmethod
- def validate_imports(self):
- pass
-
- @abc.abstractmethod
def readToGeometryRDD(cls, *args, **kwargs):
- raise NotImplementedError(f"Instance of the class {cls.__class__.__name__} has to implement method readToGeometryRDD")
\ No newline at end of file
+ raise NotImplementedError(
+ f"Instance of the class {cls.__class__.__name__} has to implement method readToGeometryRDD")
diff --git a/python/sedona/core/formatMapper/shapefileParser/shape_file_reader.py b/python/sedona/core/formatMapper/shapefileParser/shape_file_reader.py
index eb3f073..18d807b 100644
--- a/python/sedona/core/formatMapper/shapefileParser/shape_file_reader.py
+++ b/python/sedona/core/formatMapper/shapefileParser/shape_file_reader.py
@@ -21,8 +21,6 @@ from pyspark import SparkContext
from sedona.core.SpatialRDD import PolygonRDD, PointRDD, LineStringRDD
from sedona.core.SpatialRDD.spatial_rdd import SpatialRDD
from sedona.core.formatMapper.geo_reader import GeoDataReader
-from sedona.core.jvm.config import since
-from sedona.utils.decorators import require
from sedona.utils.meta import MultipleMeta
@@ -30,12 +28,6 @@ from sedona.utils.meta import MultipleMeta
class ShapefileReader(GeoDataReader, metaclass=MultipleMeta):
@classmethod
- @require(["ShapeFileReader"])
- @since("1.0.0")
- def validate_imports(cls):
- return True
-
- @classmethod
def readToGeometryRDD(cls, sc: SparkContext, inputPath: str) -> SpatialRDD:
"""
@@ -43,7 +35,6 @@ class ShapefileReader(GeoDataReader, metaclass=MultipleMeta):
:param inputPath:
:return:
"""
- ShapefileReader.validate_imports()
jvm = sc._jvm
jsc = sc._jsc
srdd = jvm.ShapefileReader.readToGeometryRDD(
@@ -63,7 +54,6 @@ class ShapefileReader(GeoDataReader, metaclass=MultipleMeta):
:param inputPath:
:return:
"""
- ShapefileReader.validate_imports()
jvm = sc._jvm
jsc = sc._jsc
srdd = jvm.ShapefileReader.readToPolygonRDD(
@@ -82,7 +72,6 @@ class ShapefileReader(GeoDataReader, metaclass=MultipleMeta):
:param inputPath:
:return:
"""
- ShapefileReader.validate_imports()
jvm = sc._jvm
jsc = sc._jsc
srdd = jvm.ShapefileReader.readToPointRDD(
@@ -101,7 +90,6 @@ class ShapefileReader(GeoDataReader, metaclass=MultipleMeta):
:param inputPath:
:return:
"""
- ShapefileReader.validate_imports()
jvm = sc._jvm
jsc = sc._jsc
srdd = jvm.ShapefileReader.readToLineStringRDD(
diff --git a/python/sedona/core/formatMapper/wkb_reader.py b/python/sedona/core/formatMapper/wkb_reader.py
index 8667610..3e9c9f6 100644
--- a/python/sedona/core/formatMapper/wkb_reader.py
+++ b/python/sedona/core/formatMapper/wkb_reader.py
@@ -25,7 +25,8 @@ from sedona.utils.meta import MultipleMeta
class WkbReader(GeoDataReader, metaclass=MultipleMeta):
@classmethod
- def readToGeometryRDD(cls, sc: SparkContext, inputPath: str, wkbColumn: int, allowInvalidGeometries: bool, skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
+ def readToGeometryRDD(cls, sc: SparkContext, inputPath: str, wkbColumn: int, allowInvalidGeometries: bool,
+ skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
"""
:param sc:
@@ -37,12 +38,14 @@ class WkbReader(GeoDataReader, metaclass=MultipleMeta):
"""
jvm = sc._jvm
spatial_rdd = SpatialRDD(sc)
- srdd = jvm.WkbReader.readToGeometryRDD(sc._jsc, inputPath, wkbColumn, allowInvalidGeometries, skipSyntacticallyInvalidGeometries)
+ srdd = jvm.WkbReader.readToGeometryRDD(sc._jsc, inputPath, wkbColumn, allowInvalidGeometries,
+ skipSyntacticallyInvalidGeometries)
spatial_rdd.set_srdd(srdd)
return spatial_rdd
@classmethod
- def readToGeometryRDD(cls, rawTextRDD: RDD, wkbColumn: int, allowInvalidGeometries: bool, skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
+ def readToGeometryRDD(cls, rawTextRDD: RDD, wkbColumn: int, allowInvalidGeometries: bool,
+ skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
"""
:param rawTextRDD:
@@ -51,12 +54,12 @@ class WkbReader(GeoDataReader, metaclass=MultipleMeta):
:param skipSyntacticallyInvalidGeometries:
:return:
"""
- WkbReader.validate_imports()
sc = rawTextRDD.ctx
jvm = sc._jvm
spatial_rdd = SpatialRDD(sc)
- srdd = jvm.WkbReader.readToGeometryRDD(rawTextRDD._jrdd, wkbColumn, allowInvalidGeometries, skipSyntacticallyInvalidGeometries)
+ srdd = jvm.WkbReader.readToGeometryRDD(rawTextRDD._jrdd, wkbColumn, allowInvalidGeometries,
+ skipSyntacticallyInvalidGeometries)
spatial_rdd.set_srdd(srdd)
- return spatial_rdd
\ No newline at end of file
+ return spatial_rdd
diff --git a/python/sedona/core/formatMapper/wkt_reader.py b/python/sedona/core/formatMapper/wkt_reader.py
index 3cc00d1..e920ec6 100644
--- a/python/sedona/core/formatMapper/wkt_reader.py
+++ b/python/sedona/core/formatMapper/wkt_reader.py
@@ -25,7 +25,8 @@ from sedona.utils.meta import MultipleMeta
class WktReader(GeoDataReader, metaclass=MultipleMeta):
@classmethod
- def readToGeometryRDD(cls, sc: SparkContext, inputPath: str, wktColumn: int, allowInvalidGeometries: bool, skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
+ def readToGeometryRDD(cls, sc: SparkContext, inputPath: str, wktColumn: int, allowInvalidGeometries: bool,
+ skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
"""
:param sc: SparkContext
@@ -37,14 +38,15 @@ class WktReader(GeoDataReader, metaclass=MultipleMeta):
"""
jvm = sc._jvm
srdd = jvm.WktReader.readToGeometryRDD(sc._jsc, inputPath, wktColumn, allowInvalidGeometries,
- skipSyntacticallyInvalidGeometries)
+ skipSyntacticallyInvalidGeometries)
spatial_rdd = SpatialRDD(sc)
spatial_rdd.set_srdd(srdd)
return spatial_rdd
@classmethod
- def readToGeometryRDD(cls, rawTextRDD: RDD, wktColumn: int, allowInvalidGeometries: bool, skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
+ def readToGeometryRDD(cls, rawTextRDD: RDD, wktColumn: int, allowInvalidGeometries: bool,
+ skipSyntacticallyInvalidGeometries: bool) -> SpatialRDD:
"""
:param rawTextRDD: RDD
diff --git a/python/sedona/core/geom/circle.py b/python/sedona/core/geom/circle.py
index 848a883..d29b692 100644
--- a/python/sedona/core/geom/circle.py
+++ b/python/sedona/core/geom/circle.py
@@ -128,4 +128,4 @@ class Circle(Polygon):
@property
def coords(self):
- raise NotImplementedError()
\ No newline at end of file
+ raise NotImplementedError()
diff --git a/python/sedona/core/jvm/abstract.py b/python/sedona/core/jvm/abstract.py
index 69d9c74..efdc218 100644
--- a/python/sedona/core/jvm/abstract.py
+++ b/python/sedona/core/jvm/abstract.py
@@ -22,7 +22,6 @@ import attr
@attr.s
class JvmObject(ABC):
-
jvm = attr.ib()
def _create_jvm_instance(self):
diff --git a/python/sedona/core/jvm/config.py b/python/sedona/core/jvm/config.py
index a0b740e..9a0e226 100644
--- a/python/sedona/core/jvm/config.py
+++ b/python/sedona/core/jvm/config.py
@@ -54,7 +54,9 @@ def since(version: str):
raise AttributeError(f"Not available before {version} sedona version")
result = function(*args, **kwargs)
return result
+
return applier
+
return wrapper
@@ -68,7 +70,9 @@ def depreciated(version: str, substitute: str):
if substitute:
logging.warning(f"Please use {substitute} instead")
return result
+
return applier
+
return wrapper
diff --git a/python/sedona/core/jvm/partitioner.py b/python/sedona/core/jvm/partitioner.py
deleted file mode 100644
index 3fe4436..0000000
--- a/python/sedona/core/jvm/partitioner.py
+++ /dev/null
@@ -1,83 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied. See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-import attr
-
-
-@attr.s
-class JvmPartitioner:
- jpart = attr.ib()
-
- def assignPartitionIds(self):
- raise NotImplementedError("Currently not supported")
-
- def assignPartitionLineage(self):
- raise NotImplementedError("Currently not supported")
-
- def dropElements(self):
- raise NotImplementedError("Currently not supported")
-
- def equals(self):
- raise NotImplementedError("Currently not supported")
-
- def findZone(self):
- raise NotImplementedError("Currently not supported")
-
- def forceGrowUp(self):
- raise NotImplementedError("Currently not supported")
-
- def getAllZones(self):
- raise NotImplementedError("Currently not supported")
-
- def getClass(self):
- raise NotImplementedError("Currently not supported")
-
- def getElements(self):
- raise NotImplementedError("Currently not supported")
-
- def getLeafZones(self):
- raise NotImplementedError("Currently not supported")
-
- def getParentZone(self):
- raise NotImplementedError("Currently not supported")
-
- def getTotalNumLeafNode(self):
- raise NotImplementedError("Currently not supported")
-
- def getZone(self):
- raise NotImplementedError("Currently not supported")
-
- def hashCode(self):
- raise NotImplementedError("Currently not supported")
-
- def insert(self):
- raise NotImplementedError("Currently not supported")
-
- def isLeaf(self):
- raise NotImplementedError("Currently not supported")
-
- def notify(self):
- raise NotImplementedError("Currently not supported")
-
- def notifyAll(self):
- raise NotImplementedError("Currently not supported")
-
- def toString(self):
- raise NotImplementedError("Currently not supported")
-
- def wait(self):
- raise NotImplementedError("Currently not supported")
diff --git a/python/sedona/core/jvm/translate.py b/python/sedona/core/jvm/translate.py
index a2a3ad8..bddc596 100644
--- a/python/sedona/core/jvm/translate.py
+++ b/python/sedona/core/jvm/translate.py
@@ -29,8 +29,8 @@ class JvmSedonaPythonConverter:
def translate_spatial_pair_rdd_to_python(self, spatial_rdd):
return self._jvm.PythonConverter.translateSpatialPairRDDToPython(spatial_rdd)
- def translate_spatial_pair_rdd_with_hashset_to_python(self, spatial_rdd):
- return self._jvm.PythonConverter.translateSpatialPairRDDWithHashSetToPython(spatial_rdd)
+ def translate_spatial_pair_rdd_with_list_to_python(self, spatial_rdd):
+ return self._jvm.PythonConverter.translateSpatialPairRDDWithListToPython(spatial_rdd)
def translate_python_rdd_to_java(self, java_rdd):
return self._jvm.PythonConverter.translatePythonRDDToJava(java_rdd)
diff --git a/python/sedona/core/spatialOperator/join_query.py b/python/sedona/core/spatialOperator/join_query.py
index b954a8e..26ce888 100644
--- a/python/sedona/core/spatialOperator/join_query.py
+++ b/python/sedona/core/spatialOperator/join_query.py
@@ -28,7 +28,8 @@ class JoinQuery:
@classmethod
@require(["JoinQuery"])
- def SpatialJoinQuery(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool, considerBoundaryIntersection: bool) -> RDD:
+ def SpatialJoinQuery(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool,
+ considerBoundaryIntersection: bool) -> RDD:
"""
:param spatialRDD: SpatialRDD
@@ -47,14 +48,15 @@ class JoinQuery:
useIndex,
considerBoundaryIntersection
)
- serialized = JvmSedonaPythonConverter(jvm)\
- .translate_spatial_pair_rdd_with_hashset_to_python(srdd)
+ serialized = JvmSedonaPythonConverter(jvm) \
+ .translate_spatial_pair_rdd_with_list_to_python(srdd)
return RDD(serialized, sc, SedonaPickler())
@classmethod
@require(["JoinQuery"])
- def DistanceJoinQuery(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool, considerBoundaryIntersection: bool) -> RDD:
+ def DistanceJoinQuery(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool,
+ considerBoundaryIntersection: bool) -> RDD:
"""
:param spatialRDD: SpatialRDD
@@ -72,8 +74,8 @@ class JoinQuery:
useIndex,
considerBoundaryIntersection
)
- serialized = JvmSedonaPythonConverter(jvm).\
- translate_spatial_pair_rdd_with_hashset_to_python(srdd)
+ serialized = JvmSedonaPythonConverter(jvm). \
+ translate_spatial_pair_rdd_with_list_to_python(srdd)
return RDD(serialized, sc, SedonaPickler())
@@ -94,14 +96,15 @@ class JoinQuery:
jvm_join_params = joinParams.jvm_instance(jvm)
srdd = jvm.JoinQuery.spatialJoin(queryWindowRDD._srdd, objectRDD._srdd, jvm_join_params)
- serialized = JvmSedonaPythonConverter(jvm).\
+ serialized = JvmSedonaPythonConverter(jvm). \
translate_spatial_pair_rdd_to_python(srdd)
return RDD(serialized, sc, SedonaPickler())
@classmethod
@require(["JoinQuery"])
- def DistanceJoinQueryFlat(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool, considerBoundaryIntersection: bool) -> RDD:
+ def DistanceJoinQueryFlat(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool,
+ considerBoundaryIntersection: bool) -> RDD:
"""
:param spatialRDD: SpatialRDD
@@ -128,7 +131,7 @@ class JoinQuery:
considerBoundaryIntersection
)
- serialized = JvmSedonaPythonConverter(jvm).\
+ serialized = JvmSedonaPythonConverter(jvm). \
translate_spatial_pair_rdd_to_python(srdd)
return RDD(serialized, sc, SedonaPickler())
@@ -136,7 +139,7 @@ class JoinQuery:
@classmethod
@require(["JoinQuery"])
def SpatialJoinQueryFlat(cls, spatialRDD: SpatialRDD, queryRDD: SpatialRDD, useIndex: bool,
- considerBoundaryIntersection: bool) -> RDD:
+ considerBoundaryIntersection: bool) -> RDD:
"""
Function takes SpatialRDD and other SpatialRDD and based on two parameters
- useIndex
@@ -167,7 +170,7 @@ class JoinQuery:
considerBoundaryIntersection
)
- serialized = JvmSedonaPythonConverter(jvm).\
+ serialized = JvmSedonaPythonConverter(jvm). \
translate_spatial_pair_rdd_to_python(srdd)
return RDD(serialized, sc, SedonaPickler())
diff --git a/python/sedona/core/spatialOperator/knn_query.py b/python/sedona/core/spatialOperator/knn_query.py
index e9ad1f9..1d4c122 100644
--- a/python/sedona/core/spatialOperator/knn_query.py
+++ b/python/sedona/core/spatialOperator/knn_query.py
@@ -31,7 +31,7 @@ class KNNQuery:
@classmethod
@require(["KNNQuery", "GeometryAdapter"])
- def SpatialKnnQuery(self, spatialRDD: SpatialRDD, originalQueryPoint: BaseGeometry, k: int, useIndex: bool):
+ def SpatialKnnQuery(self, spatialRDD: SpatialRDD, originalQueryPoint: BaseGeometry, k: int, useIndex: bool):
"""
:param spatialRDD: spatialRDD
diff --git a/python/sedona/core/spatialOperator/range_query.py b/python/sedona/core/spatialOperator/range_query.py
index cfdcd75..c8ad140 100644
--- a/python/sedona/core/spatialOperator/range_query.py
+++ b/python/sedona/core/spatialOperator/range_query.py
@@ -29,7 +29,8 @@ class RangeQuery:
@classmethod
@require(["RangeQuery", "GeometryAdapter", "GeoSerializerData"])
- def SpatialRangeQuery(self, spatialRDD: SpatialRDD, rangeQueryWindow: BaseGeometry, considerBoundaryIntersection: bool, usingIndex: bool):
+ def SpatialRangeQuery(self, spatialRDD: SpatialRDD, rangeQueryWindow: BaseGeometry,
+ considerBoundaryIntersection: bool, usingIndex: bool):
"""
:param spatialRDD:
@@ -44,7 +45,7 @@ class RangeQuery:
jvm_geom = GeometryAdapter.create_jvm_geometry_from_base_geometry(jvm, rangeQueryWindow)
- srdd = jvm.\
+ srdd = jvm. \
RangeQuery.SpatialRangeQuery(
spatialRDD._srdd,
jvm_geom,
diff --git a/python/sedona/core/utils.py b/python/sedona/core/utils.py
index 7127a84..b9990ec 100644
--- a/python/sedona/core/utils.py
+++ b/python/sedona/core/utils.py
@@ -29,4 +29,3 @@ class ImportedJvmLib:
else:
return False
return True
-
diff --git a/python/sedona/exceptions.py b/python/sedona/exceptions.py
index cab2cbf..1878860 100644
--- a/python/sedona/exceptions.py
+++ b/python/sedona/exceptions.py
@@ -1,5 +1,3 @@
-
-
# 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
diff --git a/python/sedona/register/java_libs.py b/python/sedona/register/java_libs.py
index a478318..f60eafb 100644
--- a/python/sedona/register/java_libs.py
+++ b/python/sedona/register/java_libs.py
@@ -55,4 +55,4 @@ class SedonaJvmLib(Enum):
lib = getattr(cls, geo_lib.upper())
except AttributeError:
raise AttributeError(f"{cls.__class__.__name__} has no {geo_lib} attribute")
- return lib
\ No newline at end of file
+ return lib
diff --git a/python/sedona/utils/abstract_parser.py b/python/sedona/utils/abstract_parser.py
index 2960277..2679a26 100644
--- a/python/sedona/utils/abstract_parser.py
+++ b/python/sedona/utils/abstract_parser.py
@@ -35,4 +35,3 @@ class GeometryParser(ABC):
@classmethod
def deserialize(cls, bin_parser: 'BinaryParser') -> BaseGeometry:
raise NotImplementedError("Parser has to implement deserialize method")
-
diff --git a/python/sedona/utils/binary_parser.py b/python/sedona/utils/binary_parser.py
index a23b7d5..d680421 100644
--- a/python/sedona/utils/binary_parser.py
+++ b/python/sedona/utils/binary_parser.py
@@ -47,7 +47,8 @@ class BinaryParser:
self.bytes = self._convert_to_binary_array(no_negatives)
def read_geometry(self, length: int):
- geom_bytes = b"".join([struct.pack("b", el) if el < 128 else struct.pack("b", el-256) for el in self.bytes[self.current_index: self.current_index + length]])
+ geom_bytes = b"".join([struct.pack("b", el) if el < 128 else struct.pack("b", el - 256) for el in
+ self.bytes[self.current_index: self.current_index + length]])
geom = loads(geom_bytes)
self.current_index += length
return geom
@@ -83,7 +84,7 @@ class BinaryParser:
return data
def read_string(self, length: int, encoding: str = "utf8"):
- string = self.bytes[self.current_index: self.current_index+length]
+ string = self.bytes[self.current_index: self.current_index + length]
self.current_index += length
try:
@@ -97,7 +98,7 @@ class BinaryParser:
byte_array = sc._gateway.new_array(sc._jvm.Byte, array_length)
for index, bt in enumerate(self.bytes[self.current_index: length]):
- byte_array[index] = self.bytes[self.current_index+index]
+ byte_array[index] = self.bytes[self.current_index + index]
decoded_string = sc._jvm.org.imbruced.geo_pyspark.serializers.GeoSerializerData.deserializeUserData(
byte_array
)
diff --git a/python/sedona/utils/decorators.py b/python/sedona/utils/decorators.py
index 651eec4..1b59db3 100644
--- a/python/sedona/utils/decorators.py
+++ b/python/sedona/utils/decorators.py
@@ -51,8 +51,10 @@ def require(library_names: List[str]):
if first_not_fulfill_value == -1:
return func(*args, **kwargs)
else:
- raise ModuleNotFoundError(f"Did not found {has_all_libs[first_not_fulfill_value]}, make sure that was correctly imported via py4j"
- f"Did you use SedonaRegistrator.registerAll, Your jars were properly copied to $SPARK_HOME/jars ? ")
+ raise ModuleNotFoundError(
+ f"Did not found {has_all_libs[first_not_fulfill_value]}, make sure that was correctly imported via py4j"
+ f"Did you use SedonaRegistrator.registerAll, Your jars were properly copied to $SPARK_HOME/jars ? ")
+
return run_function
- return wrapper
+ return wrapper
diff --git a/python/sedona/utils/jvm.py b/python/sedona/utils/jvm.py
index 61c6b62..d66b565 100644
--- a/python/sedona/utils/jvm.py
+++ b/python/sedona/utils/jvm.py
@@ -31,4 +31,4 @@ class JvmStorageLevel(JvmObject):
self.storage_level.useDisk, self.storage_level.useMemory,
self.storage_level.useOffHeap, self.storage_level.deserialized,
self.storage_level.replication
- )
\ No newline at end of file
+ )
diff --git a/python/sedona/utils/meta.py b/python/sedona/utils/meta.py
index c000633..dc8a723 100644
--- a/python/sedona/utils/meta.py
+++ b/python/sedona/utils/meta.py
@@ -21,6 +21,7 @@ import types
from sedona.exceptions import InvalidParametersException
from typing import Any
+
try:
from typing import GenericMeta
except ImportError:
@@ -155,6 +156,7 @@ class MultiDict(dict):
"""
Special dictionary to build multimethods in a metaclass
"""
+
def __setitem__(self, key, value):
if key in self:
# If key already exists, it must be a multimethod or callable
@@ -180,4 +182,4 @@ class MultipleMeta(type):
@classmethod
def __prepare__(cls, clsname, bases):
- return MultiDict()
\ No newline at end of file
+ return MultiDict()
diff --git a/python/sedona/utils/spatial_rdd_parser.py b/python/sedona/utils/spatial_rdd_parser.py
index 8a516b2..f897ecd 100644
--- a/python/sedona/utils/spatial_rdd_parser.py
+++ b/python/sedona/utils/spatial_rdd_parser.py
@@ -64,10 +64,10 @@ class GeoData:
if is_circle:
radius = bin_parser.read_double()
- geom = bin_parser.read_geometry(geom_bytes.__len__()-9)
+ geom = bin_parser.read_geometry(geom_bytes.__len__() - 9)
self._geom = Circle(geom, radius)
else:
- self._geom = bin_parser.read_geometry(geom_bytes.__len__()-1)
+ self._geom = bin_parser.read_geometry(geom_bytes.__len__() - 1)
self._userData = attributes["userData"]
@@ -238,4 +238,4 @@ class CircleGeometryFactory:
geom_deserializers = {
1: CircleGeometryFactory,
0: GeometryFactory
-}
\ No newline at end of file
+}
diff --git a/python/sedona/utils/types.py b/python/sedona/utils/types.py
index ffb3da2..b7a5a32 100644
--- a/python/sedona/utils/types.py
+++ b/python/sedona/utils/types.py
@@ -17,7 +17,6 @@
from typing import Union
-
numeric = Union[float, int]
path = str
crs = str
diff --git a/python/setup.py b/python/setup.py
index e49a799..74197d3 100644
--- a/python/setup.py
+++ b/python/setup.py
@@ -18,7 +18,7 @@ setup(
long_description=long_description,
long_description_content_type="text/markdown",
python_requires='>=3.6',
- install_requires=['pyspark', 'attrs', "shapely"],
+ install_requires=['pyspark<3.1.0', 'attrs', "shapely"],
project_urls={
'Bug Reports': 'https://github.com/apache/incubator-sedona'
},
diff --git a/python/tests/core/test_core_rdd.py b/python/tests/core/test_core_rdd.py
index 1f1d1bc..f2d6175 100644
--- a/python/tests/core/test_core_rdd.py
+++ b/python/tests/core/test_core_rdd.py
@@ -40,7 +40,7 @@ class TestSpatialRDD(TestBase):
point_rdd.analyze()
cnt = point_rdd.countWithoutDuplicates()
- assert cnt == 12872, f"Point RDD should have 12872 but found {cnt}"
+ assert cnt == 12796, f"Point RDD should have 12796 but found {cnt}"
def test_creating_polygon_rdd(self):
polygon_rdd = PolygonRDD(
diff --git a/python/tests/core/test_core_spatial_relations.py b/python/tests/core/test_core_spatial_relations.py
index 9ae2f2b..ebe978e 100644
--- a/python/tests/core/test_core_spatial_relations.py
+++ b/python/tests/core/test_core_spatial_relations.py
@@ -48,7 +48,7 @@ class TestJoinQuery(TestBase):
)
point_rdd.analyze()
- point_rdd.spatialPartitioning(GridType.KDBTREE)
+ point_rdd.spatialPartitioning(GridType.KDBTREE, num_partitions=10)
polygon_rdd.spatialPartitioning(point_rdd.getPartitioner())
result = JoinQuery.SpatialJoinQuery(
point_rdd,
@@ -57,4 +57,4 @@ class TestJoinQuery(TestBase):
False
)
- print(result.count())
\ No newline at end of file
+ assert result.count() == 26
diff --git a/python/tests/core/test_spatial_rdd_from_disc.py b/python/tests/core/test_spatial_rdd_from_disc.py
index fab6833..7d7ca2f 100644
--- a/python/tests/core/test_spatial_rdd_from_disc.py
+++ b/python/tests/core/test_spatial_rdd_from_disc.py
@@ -152,8 +152,8 @@ class TestDiscUtils(TestBase):
linestring_rdd.analyze()
print(linestring_rdd.boundaryEnvelope)
- linestring_rdd.spatialPartitioning(GridType.RTREE)
- polygon_rdd.spatialPartitioning(linestring_rdd.grids)
+ linestring_rdd.spatialPartitioning(GridType.KDBTREE)
+ polygon_rdd.spatialPartitioning(linestring_rdd.getPartitioner())
polygon_rdd.buildIndex(IndexType.RTREE, True)
linestring_rdd.buildIndex(IndexType.RTREE, True)
diff --git a/python/tests/properties/crs_transform.py b/python/tests/properties/crs_transform.py
index d77fa2f..265561b 100644
--- a/python/tests/properties/crs_transform.py
+++ b/python/tests/properties/crs_transform.py
@@ -19,14 +19,13 @@ import os
from shapely.geometry import Point
-from sedona.core.enums import FileDataSplitter, GridType, IndexType
+from sedona.core.enums import FileDataSplitter, IndexType
from sedona.core.geom.envelope import Envelope
from tests.tools import tests_path
input_location = os.path.join(tests_path, "resources/crs-test-point.csv")
offset = 0
splitter = FileDataSplitter.CSV
-grid_type = GridType.RTREE
index_type = IndexType.RTREE
num_partitions = 11
distance = 0.01
diff --git a/python/tests/properties/linestring_properties.py b/python/tests/properties/linestring_properties.py
index 502e7f7..0ae2b25 100644
--- a/python/tests/properties/linestring_properties.py
+++ b/python/tests/properties/linestring_properties.py
@@ -25,7 +25,7 @@ input_location = os.path.join(tests_path, "resources/primaryroads-linestring.csv
query_window_set = os.path.join(tests_path, "resources/zcta510-small.csv")
offset = 0
splitter = FileDataSplitter.CSV
-grid_type = "rtree"
+grid_type = "kdbtree"
index_type = "rtree"
num_partitions = 5
distance = 0.01
diff --git a/python/tests/properties/polygon_properties.py b/python/tests/properties/polygon_properties.py
index c1c776b..28860d5 100644
--- a/python/tests/properties/polygon_properties.py
+++ b/python/tests/properties/polygon_properties.py
@@ -25,12 +25,13 @@ input_location = os.path.join(tests_path, "resources/primaryroads-polygon.csv")
query_window_set = os.path.join(tests_path, "resources/zcta510-small.csv")
offset = 0
splitter = FileDataSplitter.CSV
-grid_type = "rtree"
+grid_type = "kdbtree"
index_type = "rtree"
num_partitions = 5
distance = 0.01
input_location_query_polygon = os.path.join(tests_path, "resources/crs-test-polygon.csv")
query_polygon_count = 13361
+
query_envelope = Envelope(14313844.294334238, 16802290.853830762, 942450.5989896103, 8631908.270651892)
query_polygon_set = os.path.join(tests_path, "resources/primaryroads-polygon.csv")
input_location_geo_json = os.path.join(tests_path, "resources/testPolygon.json")
@@ -48,4 +49,4 @@ polygon_rdd_splitter = FileDataSplitter.CSV
polygon_rdd_index_type = IndexType.RTREE
polygon_rdd_num_partitions = 5
polygon_rdd_start_offset = 0
-polygon_rdd_end_offset = 9
\ No newline at end of file
+polygon_rdd_end_offset = 9
diff --git a/python/tests/resources/points.csv b/python/tests/resources/points.csv
index 6478670..d1b3ed3 100644
--- a/python/tests/resources/points.csv
+++ b/python/tests/resources/points.csv
@@ -12871,4 +12871,4 @@ Soft Eis und Hot-Dog" POINT (14.5507035 52.3432844)
6823721834 2903 drinking_water POINT (15.1283999 51.9953406)
6823721838 2701 tourist_info ↑ Nad jeziorami 85 POINT (15.1300532 51.9950438)
6823772928 2701 tourist_info Ogłoszenia POINT (15.1323097 51.9955243)
-6823823445 2204 park Stara wojskowa strzelnica POINT (15.193786 51.6616318)
+6823823445 2204 park Stara wojskowa strzelnica POINT (15.193786 51.6616318)
\ No newline at end of file
diff --git a/python/tests/spatial_operator/test_join_base.py b/python/tests/spatial_operator/test_join_base.py
index 0ae9164..434e987 100644
--- a/python/tests/spatial_operator/test_join_base.py
+++ b/python/tests/spatial_operator/test_join_base.py
@@ -53,15 +53,9 @@ class TestJoinBase(TestBase):
rdd.rawJvmSpatialRDD, StorageLevel.MEMORY_ONLY
)
- def partition_rdds(self, query_rdd: SpatialRDD, spatial_rdd: SpatialRDD, grid_type, use_legacy_apis):
+ def partition_rdds(self, query_rdd: SpatialRDD, spatial_rdd: SpatialRDD, grid_type):
spatial_rdd.spatialPartitioning(grid_type)
- if use_legacy_apis:
- if grid_type != GridType.QUADTREE:
- query_rdd.spatialPartitioning(spatial_rdd.grids)
- else:
- query_rdd.spatialPartitioning(spatial_rdd.partitionTree)
- else:
- query_rdd.spatialPartitioning(spatial_rdd.getPartitioner())
+ return query_rdd.spatialPartitioning(spatial_rdd.getPartitioner())
def expect_to_preserve_original_duplicates(self, grid_type):
return grid_type == GridType.QUADTREE or grid_type == GridType.KDBTREE
@@ -75,14 +69,10 @@ class TestJoinBase(TestBase):
def sanity_check_join_results(self, results):
for raw_data in results:
- assert raw_data[0].getUserData() is not None
assert raw_data[1].__len__()
for geo_data in raw_data[1]:
- assert geo_data.getUserData() is not None
assert raw_data[0].geom.intersects(geo_data.geom)
def sanity_check_flat_join_results(self, results):
for row_data in results:
- assert row_data[0].getUserData() is not None
- assert row_data[1].getUserData() is not None
assert row_data[0].geom.intersects(row_data[1].geom)
\ No newline at end of file
diff --git a/python/tests/spatial_operator/test_join_query_correctness.py b/python/tests/spatial_operator/test_join_query_correctness.py
index a31a1a1..79e7aeb 100644
--- a/python/tests/spatial_operator/test_join_query_correctness.py
+++ b/python/tests/spatial_operator/test_join_query_correctness.py
@@ -185,7 +185,7 @@ class TestJoinQueryCorrectness(TestBase):
@classmethod
def verify_join_result(cls, result):
- assert 200 == result.__len__()
+ assert result.__len__() == 200
@classmethod
def make_square(cls, minx: float, miny: float, side: float) -> Polygon:
diff --git a/python/tests/spatial_operator/test_linestring_join.py b/python/tests/spatial_operator/test_linestring_join.py
index 58956d5..e6b7f66 100644
--- a/python/tests/spatial_operator/test_linestring_join.py
+++ b/python/tests/spatial_operator/test_linestring_join.py
@@ -42,11 +42,9 @@ def pytest_generate_tests(metafunc):
parameters = [
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.KDBTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.KDBTREE),
]
params_dyn = [{**param, **{"index_type": IndexType.QUADTREE}} for param in parameters]
params_dyn.extend([{**param, **{"index_type": IndexType.RTREE}} for param in parameters])
@@ -59,23 +57,26 @@ class TestRectangleJoin(TestJoinBase):
"test_index_int": params_dyn
}
- def test_nested_loop(self, num_partitions, use_legacy_apis, grid_type):
+ def test_nested_loop(self, num_partitions, grid_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_linestring_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, True).collect()
self.sanity_check_join_results(result)
- assert match_count == self.count_join_results(result)
+ expected_count = match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else match_count
- def test_dynamic_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type):
+ assert expected_count == self.count_join_results(result)
+
+ def test_dynamic_index_int(self, num_partitions, grid_type, index_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_linestring_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
join_params = JoinParams(True, index_type, JoinBuildSide.LEFT)
result = JoinQuery.spatialJoin(query_rdd, spatial_rdd, join_params).collect()
@@ -87,15 +88,17 @@ class TestRectangleJoin(TestJoinBase):
assert expected_count == result.__len__()
- def test_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type):
+ def test_index_int(self, num_partitions, grid_type, index_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_linestring_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
spatial_rdd.buildIndex(index_type, True)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, True).collect()
self.sanity_check_join_results(result)
- assert match_count == self.count_join_results(result)
+ expected_count = match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else match_count
+ assert expected_count == self.count_join_results(result)
diff --git a/python/tests/spatial_operator/test_point_join.py b/python/tests/spatial_operator/test_point_join.py
index e2130ce..01db0c2 100644
--- a/python/tests/spatial_operator/test_point_join.py
+++ b/python/tests/spatial_operator/test_point_join.py
@@ -27,15 +27,15 @@ from tests.tools import tests_path
input_location = os.path.join(tests_path, "resources/arealm-small.csv")
input_location_query_window = os.path.join(tests_path, "resources/zcta510-small.csv")
-offset=1
-splitter=FileDataSplitter.CSV
-numPartitions=11
-distance=0.01
+offset = 1
+splitter = FileDataSplitter.CSV
+numPartitions = 11
+distance = 0.01
query_polygon_set = os.path.join(tests_path, "resources/primaryroads-polygon.csv")
-inputCount=3000
-inputBoundary=-173.120769, -84.965961, 30.244859, 71.355134
-rectangle_match_count=103
-rectangle_match_with_original_duplicates_count=103
+inputCount = 3000
+inputBoundary = -173.120769, -84.965961, 30.244859, 71.355134
+rectangle_match_count = 103
+rectangle_match_with_original_duplicates_count = 103
polygon_match_count = 472
polygon_match_with_original_duplicates_count = 562
@@ -49,11 +49,9 @@ def pytest_generate_tests(metafunc):
parameters = [
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.KDBTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.KDBTREE),
]
@@ -70,19 +68,21 @@ class TestRectangleJoin(TestJoinBase):
"test_dynamic_r_tree_with_polygons": parameters
}
- def test_nested_loop_with_rectangles(self, num_partitions, grid_type, use_legacy_apis):
+ def test_nested_loop_with_rectangles(self, num_partitions, grid_type):
query_rdd = self.create_rectangle_rdd(input_location_query_window, splitter, num_partitions)
- self.nested_loop(query_rdd, num_partitions, grid_type, use_legacy_apis, rectangle_match_count)
+ self.nested_loop(query_rdd, num_partitions, grid_type, rectangle_match_count)
- def test_nested_loop_with_polygons(self, num_partitions, grid_type, use_legacy_apis):
+ def test_nested_loop_with_polygons(self, num_partitions, grid_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
- self.nested_loop(query_rdd, num_partitions, grid_type, use_legacy_apis, polygon_match_count)
+ expected_count = polygon_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else polygon_match_count
+ self.nested_loop(query_rdd, num_partitions, grid_type, expected_count)
- def nested_loop(self, query_rdd, num_partitions, grid_type, use_legacy_apis, expected_count):
+ def nested_loop(self, query_rdd, num_partitions, grid_type, expected_count):
spatial_rdd = self.create_point_rdd(input_location, splitter, num_partitions)
self.partition_rdds(
- query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ query_rdd, spatial_rdd, grid_type)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, True).collect()
@@ -90,38 +90,42 @@ class TestRectangleJoin(TestJoinBase):
self.sanity_check_join_results(result)
assert expected_count == self.count_join_results(result)
- def test_rtree_with_rectangles(self, num_partitions, use_legacy_apis, grid_type):
+ def test_rtree_with_rectangles(self, num_partitions, grid_type):
query_rdd = self.create_rectangle_rdd(input_location_query_window, splitter, num_partitions)
self.index_int(
- query_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.RTREE, polygon_match_count
+ query_rdd, num_partitions, grid_type, IndexType.RTREE, polygon_match_count
)
- def test_r_tree_with_polygons(self, num_partitions, use_legacy_apis, grid_type):
+ def test_r_tree_with_polygons(self, num_partitions, grid_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
+ expected_count = polygon_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else polygon_match_count
self.index_int(
- query_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.RTREE, polygon_match_count
+ query_rdd, num_partitions, grid_type, IndexType.RTREE, expected_count
)
- def test_quad_tree_with_rectangles(self, num_partitions, use_legacy_apis, grid_type):
+ def test_quad_tree_with_rectangles(self, num_partitions, grid_type):
query_rdd = self.create_rectangle_rdd(input_location_query_window, splitter, num_partitions)
self.index_int(
- query_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.QUADTREE, polygon_match_count
+ query_rdd, num_partitions, grid_type, IndexType.QUADTREE, polygon_match_count
)
- def test_quad_tree_with_polygons(self, num_partitions, use_legacy_apis, grid_type):
+ def test_quad_tree_with_polygons(self, num_partitions, grid_type):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
+ expected_count = polygon_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else polygon_match_count
self.index_int(
- query_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.QUADTREE, polygon_match_count
+ query_rdd, num_partitions, grid_type, IndexType.QUADTREE, expected_count
)
- def index_int(self, query_rdd, num_partitions, use_legacy_apis, grid_type, index_type, expected_count):
+ def index_int(self, query_rdd, num_partitions, grid_type, index_type, expected_count):
spatial_rdd = self.create_point_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
spatial_rdd.buildIndex(index_type, True)
result = JoinQuery.SpatialJoinQuery(
@@ -130,22 +134,24 @@ class TestRectangleJoin(TestJoinBase):
self.sanity_check_join_results(result)
assert expected_count, self.count_join_results(result)
- def test_dynamic_r_tree_with_rectangles(self, grid_type, num_partitions, use_legacy_apis):
+ def test_dynamic_r_tree_with_rectangles(self, grid_type, num_partitions):
polygon_rdd = self.create_rectangle_rdd(input_location_query_window, splitter, num_partitions)
- expected_count = rectangle_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(grid_type) \
+ expected_count = rectangle_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) \
else rectangle_match_count
- self.dynamic_rtree_int(polygon_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.RTREE, expected_count)
+ self.dynamic_rtree_int(polygon_rdd, num_partitions, grid_type, IndexType.RTREE, expected_count)
- def test_dynamic_r_tree_with_polygons(self, grid_type, num_partitions, use_legacy_apis):
+ def test_dynamic_r_tree_with_polygons(self, grid_type, num_partitions):
polygon_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
- expected_count = polygon_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(grid_type) \
+ expected_count = polygon_match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) \
else polygon_match_count
- self.dynamic_rtree_int(polygon_rdd, num_partitions, use_legacy_apis, grid_type, IndexType.RTREE, expected_count)
+ self.dynamic_rtree_int(polygon_rdd, num_partitions, grid_type, IndexType.RTREE, expected_count)
- def dynamic_rtree_int(self, query_rdd, num_partitions, use_legacy_apis, grid_type, index_type, expected_count):
+ def dynamic_rtree_int(self, query_rdd, num_partitions, grid_type, index_type, expected_count):
spatial_rdd = self.create_point_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
join_params = JoinParams(True, index_type, JoinBuildSide.LEFT)
results = JoinQuery.spatialJoin(query_rdd, spatial_rdd, join_params).collect()
diff --git a/python/tests/spatial_operator/test_point_knn.py b/python/tests/spatial_operator/test_point_knn.py
index bf110d0..3496250 100644
--- a/python/tests/spatial_operator/test_point_knn.py
+++ b/python/tests/spatial_operator/test_point_knn.py
@@ -44,7 +44,6 @@ class TestPointKnn(TestBase):
for i in range(self.loop_times):
result = KNNQuery.SpatialKnnQuery(point_rdd, self.query_point, self.top_k, False)
assert result.__len__() > -1
- assert result[0].getUserData() is not None
def test_spatial_knn_query_using_index(self):
point_rdd = PointRDD(self.sc, input_location, offset, splitter, False)
@@ -53,7 +52,6 @@ class TestPointKnn(TestBase):
for i in range(self.loop_times):
result = KNNQuery.SpatialKnnQuery(point_rdd, self.query_point, self.top_k, False)
assert result.__len__() > -1
- assert result[0].getUserData() is not None
def test_spatial_knn_correctness(self):
point_rdd = PointRDD(self.sc, input_location, offset, splitter, False)
diff --git a/python/tests/spatial_operator/test_point_range.py b/python/tests/spatial_operator/test_point_range.py
index 17c327f..5331ac6 100644
--- a/python/tests/spatial_operator/test_point_range.py
+++ b/python/tests/spatial_operator/test_point_range.py
@@ -57,9 +57,6 @@ class TestPointRange(TestBase):
SpatialRangeQuery(spatial_rdd, self.query_envelope, False, False)\
.count()
assert result_size == 2830
- assert RangeQuery.SpatialRangeQuery(
- spatial_rdd, self.query_envelope, False, False).take(10)[1].\
- getUserData() is not None
def test_spatial_range_query_using_index(self):
spatial_rdd = PointRDD(self.sc, input_location, offset, splitter, False)
@@ -71,6 +68,3 @@ class TestPointRange(TestBase):
SpatialRangeQuery(spatial_rdd, self.query_envelope, False, False)\
.count()
assert result_size == 2830
- assert RangeQuery.SpatialRangeQuery(
- spatial_rdd, self.query_envelope, False, False).take(10)[1].\
- getUserData() is not None
diff --git a/python/tests/spatial_operator/test_polygon_join.py b/python/tests/spatial_operator/test_polygon_join.py
index 3d91db2..2bb01fc 100644
--- a/python/tests/spatial_operator/test_polygon_join.py
+++ b/python/tests/spatial_operator/test_polygon_join.py
@@ -25,7 +25,6 @@ from sedona.core.spatialOperator.join_params import JoinParams
from tests.spatial_operator.test_join_base import TestJoinBase
from tests.tools import tests_path
-
input_location = os.path.join(tests_path, "resources/primaryroads-polygon.csv")
query_window_set = os.path.join(tests_path, "resources/zcta510-small.csv")
query_polygon_set = os.path.join(tests_path, "resources/primaryroads-polygon.csv")
@@ -45,16 +44,11 @@ def pytest_generate_tests(metafunc):
parameters = [
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.RTREE, intersects=False),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.RTREE, intersects=False),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.QUADTREE, intersects=False),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.QUADTREE, intersects=False),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.KDBTREE, intersects=False),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.RTREE, intersects=True),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.RTREE, intersects=True),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.QUADTREE, intersects=True),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.QUADTREE, intersects=True),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.KDBTREE, intersects=True)
+ dict(num_partitions=11, grid_type=GridType.QUADTREE, intersects=False),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE, intersects=False),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE, intersects=True),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE, intersects=True),
+ dict(num_partitions=11, grid_type=GridType.KDBTREE, intersects=True)
]
params_dyn = [{**param, **{"index_type": IndexType.QUADTREE}} for param in parameters]
params_dyn.extend([{**param, **{"index_type": IndexType.RTREE}} for param in parameters])
@@ -67,23 +61,23 @@ class TestRectangleJoin(TestJoinBase):
"test_index_int": params_dyn
}
- def test_nested_loop(self, num_partitions, use_legacy_apis, grid_type, intersects):
+ def test_nested_loop(self, num_partitions, grid_type, intersects):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_polygon_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, intersects).collect()
self.sanity_check_join_results(result)
- assert self.get_expected_count(intersects) == self.count_join_results(result)
+ assert self.get_expected_with_original_duplicates_count(intersects) == self.count_join_results(result)
- def test_dynamic_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type, intersects):
+ def test_dynamic_index_int(self, num_partitions, grid_type, index_type, intersects):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_polygon_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
join_params = JoinParams(intersects, index_type, JoinBuildSide.LEFT)
result = JoinQuery.spatialJoin(query_rdd, spatial_rdd, join_params).collect()
@@ -94,18 +88,18 @@ class TestRectangleJoin(TestJoinBase):
if self.expect_to_preserve_original_duplicates(grid_type) else self.get_expected_count(intersects)
assert expected_count == result.__len__()
- def test_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type, intersects):
+ def test_index_int(self, num_partitions, grid_type, index_type, intersects):
query_rdd = self.create_polygon_rdd(query_polygon_set, splitter, num_partitions)
spatial_rdd = self.create_polygon_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
spatial_rdd.buildIndex(index_type, True)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, True, intersects).collect()
self.sanity_check_join_results(result)
- assert self.get_expected_count(intersects) == self.count_join_results(result)
+ assert self.get_expected_with_original_duplicates_count(intersects) == self.count_join_results(result)
def get_expected_count(self, intersects):
return intersects_match_count if intersects else contains_match_count
diff --git a/python/tests/spatial_operator/test_rectangle_join.py b/python/tests/spatial_operator/test_rectangle_join.py
index 4ff83cb..1e1ac30 100644
--- a/python/tests/spatial_operator/test_rectangle_join.py
+++ b/python/tests/spatial_operator/test_rectangle_join.py
@@ -46,11 +46,9 @@ def pytest_generate_tests(metafunc):
parameters = [
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.RTREE),
- dict(num_partitions=11, use_legacy_apis=True, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.QUADTREE),
- dict(num_partitions=11, use_legacy_apis=False, grid_type=GridType.KDBTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.QUADTREE),
+ dict(num_partitions=11, grid_type=GridType.KDBTREE),
]
params_dyn = [{**param, **{"index_type": IndexType.QUADTREE}} for param in parameters]
params_dyn.extend([{**param, **{"index_type": IndexType.RTREE}} for param in parameters])
@@ -63,11 +61,11 @@ class TestRectangleJoin(TestJoinBase):
"test_index_int": params_dyn
}
- def test_nested_loop(self, num_partitions, use_legacy_apis, grid_type):
+ def test_nested_loop(self, num_partitions, grid_type):
query_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
spatial_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, True).collect()
@@ -76,13 +74,15 @@ class TestRectangleJoin(TestJoinBase):
for el in result:
count += el[1].__len__()
self.sanity_check_join_results(result)
- assert match_count == self.count_join_results(result)
+ expected_count = match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else match_count
+ assert expected_count == self.count_join_results(result)
- def test_dynamic_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type):
+ def test_dynamic_index_int(self, num_partitions, grid_type, index_type):
query_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
spatial_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
join_params = JoinParams(True, index_type, JoinBuildSide.LEFT)
result = JoinQuery.spatialJoin(query_rdd, spatial_rdd, join_params).collect()
@@ -94,15 +94,18 @@ class TestRectangleJoin(TestJoinBase):
assert expected_count == result.__len__()
- def test_index_int(self, num_partitions, use_legacy_apis, grid_type, index_type):
+ def test_index_int(self, num_partitions, grid_type, index_type):
query_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
spatial_rdd = self.create_rectangle_rdd(input_location, splitter, num_partitions)
- self.partition_rdds(query_rdd, spatial_rdd, grid_type, use_legacy_apis)
+ self.partition_rdds(query_rdd, spatial_rdd, grid_type)
spatial_rdd.buildIndex(index_type, True)
result = JoinQuery.SpatialJoinQuery(
spatial_rdd, query_rdd, False, True).collect()
self.sanity_check_join_results(result)
- assert match_count == self.count_join_results(result)
+ expected_count = match_with_original_duplicates_count if self.expect_to_preserve_original_duplicates(
+ grid_type) else match_count
+
+ assert expected_count == self.count_join_results(result)
diff --git a/python/tests/spatial_rdd/test_linestring_rdd.py b/python/tests/spatial_rdd/test_linestring_rdd.py
index 29d3921..f693e88 100644
--- a/python/tests/spatial_rdd/test_linestring_rdd.py
+++ b/python/tests/spatial_rdd/test_linestring_rdd.py
@@ -146,51 +146,6 @@ class TestLineStringRDD(TestBase):
spatial_rdd_copy.rawJvmSpatialRDD = spatial_rdd.rawJvmSpatialRDD
spatial_rdd_copy.analyze()
- def test_hilbert_curve_spatial_partitioning(self):
- spatial_rdd = LineStringRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.HILBERT)
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- def test_rtree_spatial_partitioning(self):
- spatial_rdd = LineStringRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.RTREE)
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- def test_voronoi_spatial_partitioning(self):
- spatial_rdd = LineStringRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.VORONOI)
- for envelope in spatial_rdd.grids:
- print(envelope)
-
def test_build_index_without_set_grid(self):
spatial_rdd = LineStringRDD(
sparkContext=self.sc,
diff --git a/python/tests/spatial_rdd/test_point_rdd.py b/python/tests/spatial_rdd/test_point_rdd.py
index 681e26f..24b7bdf 100644
--- a/python/tests/spatial_rdd/test_point_rdd.py
+++ b/python/tests/spatial_rdd/test_point_rdd.py
@@ -110,64 +110,7 @@ class TestPointRDD(TestBase):
newLevel=StorageLevel.MEMORY_ONLY
)
spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.EQUALGRID)
-
- for envelope in spatial_rdd.grids:
- print("PointRDD spatial partitioning grids: " + str(envelope))
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
- def test_hilbert_curve_spatial_partitioning(self):
- spatial_rdd = PointRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- Offset=offset,
- splitter=splitter,
- carryInputData=False,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.HILBERT)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
- def test_r_tree_spatial_partitioning(self):
- spatial_rdd = PointRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- Offset=offset,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.RTREE)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
- def test_voronoi_spatial_partitioning(self):
- spatial_rdd = PointRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- Offset=offset,
- splitter=splitter,
- carryInputData=False,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.VORONOI)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
+ spatial_rdd.spatialPartitioning(GridType.QUADTREE)
assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
diff --git a/python/tests/spatial_rdd/test_polygon_rdd.py b/python/tests/spatial_rdd/test_polygon_rdd.py
index 9c7825e..61b8528 100644
--- a/python/tests/spatial_rdd/test_polygon_rdd.py
+++ b/python/tests/spatial_rdd/test_polygon_rdd.py
@@ -294,52 +294,6 @@ class TestPolygonRDD(TestBase):
assert spatial_rdd.boundaryEnvelope is not None
assert spatial_rdd.rawSpatialRDD.take(1)[0].getUserData() == "31\t039\t00835841\t31039\tCuming\tCuming County\t06\tH1\tG4020\t\t\t\tA\t1477895811\t10447360\t+41.9158651\t-096.7885168"
- def test_hilbert_curve_spatial_partitioning(self):
- spatial_rdd = PolygonRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.HILBERT)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- def test_r_tree_spatial_partitioning(self):
- spatial_rdd = PolygonRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.RTREE)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- def test_voronoi_spatial_partitioning(self):
- spatial_rdd = PolygonRDD(
- sparkContext=self.sc,
- InputLocation=input_location,
- splitter=FileDataSplitter.CSV,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.VORONOI)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
-
def test_build_index_without_set_grid(self):
spatial_rdd = PolygonRDD(
self.sc,
diff --git a/python/tests/spatial_rdd/test_rectangle_rdd.py b/python/tests/spatial_rdd/test_rectangle_rdd.py
index 693e2cb..67619c8 100644
--- a/python/tests/spatial_rdd/test_rectangle_rdd.py
+++ b/python/tests/spatial_rdd/test_rectangle_rdd.py
@@ -92,62 +92,6 @@ class TestRectangleRDD(TestBase):
spatial_rdd_copy.rawJvmSpatialRDD = spatial_rdd.rawJvmSpatialRDD
spatial_rdd_copy.analyze()
- def test_hilbert_curve_spatial_partitioning(self):
- spatial_rdd = RectangleRDD(
- sparkContext=self.sc,
- InputLocation=inputLocation,
- Offset=offset,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
- spatial_rdd.analyze()
-
- spatial_rdd.spatialPartitioning(GridType.HILBERT)
-
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
- def test_rtree_spatial_partitioning(self):
- spatial_rdd = RectangleRDD(
- sparkContext=self.sc,
- InputLocation=inputLocation,
- Offset=offset,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
-
- spatial_rdd.analyze()
-
- spatial_rdd.spatialPartitioning(GridType.RTREE)
-
- for envelope in spatial_rdd.grids:
- print(spatial_rdd)
-
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
- def test_voronoi_spatial_partitioning(self):
- spatial_rdd = RectangleRDD(
- sparkContext=self.sc,
- InputLocation=inputLocation,
- Offset=offset,
- splitter=splitter,
- carryInputData=True,
- partitions=10,
- newLevel=StorageLevel.MEMORY_ONLY
- )
- spatial_rdd.analyze()
- spatial_rdd.spatialPartitioning(GridType.VORONOI)
- for envelope in spatial_rdd.grids:
- print(envelope)
-
- assert spatial_rdd.countWithoutDuplicates() == spatial_rdd.countWithoutDuplicatesSPRDD()
-
def test_build_index_without_set_grid(self):
spatial_rdd = RectangleRDD(
sparkContext=self.sc,
diff --git a/python/tests/spatial_rdd/test_spatial_rdd.py b/python/tests/spatial_rdd/test_spatial_rdd.py
index 485fe60..a0ab7f2 100644
--- a/python/tests/spatial_rdd/test_spatial_rdd.py
+++ b/python/tests/spatial_rdd/test_spatial_rdd.py
@@ -210,19 +210,6 @@ class TestSpatialRDD(TestBase):
for grid_type in GridType:
spatial_rdd = self.create_spatial_rdd()
spatial_rdd.spatialPartitioning(grid_type)
- grids = spatial_rdd.grids
- if grid_type != GridType.QUADTREE and grid_type != GridType.KDBTREE:
- assert grids is not None
- assert isinstance(grids, list)
- assert isinstance(grids[0], Envelope)
- else:
- assert grids is None
-
- def test_indexed_rdd(self):
- pass
-
- def test_indexed_raw_rdd(self):
- pass
def test_partition_tree(self):
spatial_rdd = self.create_spatial_rdd()
@@ -231,4 +218,4 @@ class TestSpatialRDD(TestBase):
spatial_rdd.spatialPartitioning(GridType.QUADTREE)
- print(spatial_rdd.partitionTree)
+ print(spatial_rdd.getPartitioner())
diff --git a/python/tests/spatial_rdd/test_spatial_rdd_writer.py b/python/tests/spatial_rdd/test_spatial_rdd_writer.py
index aa04b4b..bc2b37a 100644
--- a/python/tests/spatial_rdd/test_spatial_rdd_writer.py
+++ b/python/tests/spatial_rdd/test_spatial_rdd_writer.py
@@ -57,7 +57,7 @@ rectangleMatchWithOriginalDuplicatesCount = 103
polygonMatchCount = 472
polygonMatchWithOriginalDuplicatesCount = 562
-
+## todo add missing tests
def remove_directory(path: str) -> bool:
try:
shutil.rmtree(path)
diff --git a/python/tests/utils/test_crs_transformation.py b/python/tests/utils/test_crs_transformation.py
index da15a0a..9c7748b 100644
--- a/python/tests/utils/test_crs_transformation.py
+++ b/python/tests/utils/test_crs_transformation.py
@@ -18,10 +18,11 @@
from pyspark import StorageLevel
from sedona.core.SpatialRDD import PointRDD, PolygonRDD, CircleRDD
+from sedona.core.enums import GridType
from sedona.core.geom.circle import Circle
from sedona.core.spatialOperator import RangeQuery, KNNQuery, JoinQuery
-from sedona.utils.adapter import Adapter
from tests.properties.crs_transform import *
+from tests.properties.polygon_properties import grid_type
from tests.test_base import TestBase
from tests.tools import distance_sorting_functions
@@ -119,8 +120,7 @@ class TestCrsTransformation(TestBase):
StorageLevel.MEMORY_ONLY, "epsg:4326", "epsg:3005"
)
spatial_rdd.spatialPartitioning(grid_type)
-
- query_rdd.spatialPartitioning(spatial_rdd.grids)
+ query_rdd.spatialPartitioning(spatial_rdd.getPartitioner())
result = JoinQuery.SpatialJoinQuery(spatial_rdd, query_rdd, False, True).collect()
assert result[1][0].getUserData() is not None
@@ -142,11 +142,14 @@ class TestCrsTransformation(TestBase):
StorageLevel.MEMORY_ONLY, "epsg:4326", "epsg:3005"
)
+ query_rdd.analyze()
+ spatial_rdd.analyze()
+
spatial_rdd.spatialPartitioning(grid_type)
spatial_rdd.buildIndex(IndexType.RTREE, True)
- query_rdd.spatialPartitioning(spatial_rdd.grids)
+ query_rdd.spatialPartitioning(spatial_rdd.getPartitioner())
result = JoinQuery.SpatialJoinQuery(spatial_rdd, query_rdd, False, True).collect()
@@ -170,12 +173,12 @@ class TestCrsTransformation(TestBase):
"epsg:4326", "epsg:3857")
object_rdd.rawJvmSpatialRDD.jsrdd.repartition(4)
- object_rdd.spatialPartitioning(GridType.RTREE)
+ object_rdd.spatialPartitioning(GridType.KDBTREE)
object_rdd.buildIndex(IndexType.RTREE, True)
- window_rdd.spatialPartitioning(object_rdd.grids)
+ window_rdd.spatialPartitioning(object_rdd.getPartitioner())
results = JoinQuery.DistanceJoinQuery(object_rdd, window_rdd, True, False).collect()
- assert results.__len__() == 5467
+ assert 5467 == results.__len__()
for data in results:
for polygon_data in data[1]:
diff --git a/sql/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala b/sql/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
index a37e89d..cfaf077 100644
--- a/sql/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
+++ b/sql/src/main/scala/org/apache/sedona/sql/utils/Adapter.scala
@@ -19,6 +19,7 @@
package org.apache.sedona.sql.utils
import org.apache.sedona.core.spatialRDD.SpatialRDD
+import org.apache.sedona.core.utils.GeomUtils
import org.apache.spark.api.java.{JavaPairRDD, JavaRDD}
import org.apache.spark.rdd.RDD
import org.locationtech.jts.geom.Geometry
@@ -118,7 +119,7 @@ object Adapter {
}
def toDf[T <: Geometry](spatialRDD: SpatialRDD[T], fieldNames: List[String], sparkSession: SparkSession): DataFrame = {
- val rowRdd = spatialRDD.rawSpatialRDD.rdd.map[Row](f => Row.fromSeq(f.toString.split("\t", -1).toSeq))
+ val rowRdd = spatialRDD.rawSpatialRDD.rdd.map[Row](f => Row.fromSeq(GeomUtils.printGeom(f).split("\t", -1).toSeq))
if (fieldNames != null && fieldNames.nonEmpty) {
var fieldArray = new Array[StructField](fieldNames.size + 1)
fieldArray(0) = StructField("geometry", StringType)
@@ -137,8 +138,8 @@ object Adapter {
def toDf(spatialPairRDD: JavaPairRDD[Geometry, Geometry], sparkSession: SparkSession): DataFrame = {
val rowRdd = spatialPairRDD.rdd.map[Row](f => {
- val seq1 = f._1.toString.split("\t").toSeq
- val seq2 = f._2.toString.split("\t").toSeq
+ val seq1 = GeomUtils.printGeom(f._1).split("\t").toSeq
+ val seq2 = GeomUtils.printGeom(f._2).split("\t").toSeq
val result = seq1 ++ seq2
Row.fromSeq(result)
})
@@ -154,8 +155,8 @@ object Adapter {
def toDf(spatialPairRDD: JavaPairRDD[Geometry, Geometry], leftFieldnames: List[String], rightFieldNames: List[String], sparkSession: SparkSession): DataFrame = {
val rowRdd = spatialPairRDD.rdd.map[Row](f => {
- val seq1 = f._1.toString.split("\t").toSeq
- val seq2 = f._2.toString.split("\t").toSeq
+ val seq1 = GeomUtils.printGeom(f._1).split("\t").toSeq
+ val seq2 = GeomUtils.printGeom(f._2).split("\t").toSeq
val result = seq1 ++ seq2
Row.fromSeq(result)
})
@@ -174,8 +175,9 @@ object Adapter {
dataFrame.rdd.map[Geometry](f => {
var geometry = f.get(geometryColId).asInstanceOf[Geometry]
var fieldSize = f.size
- var userData = ""
+ var userData:String = null
if (fieldSize > 1) {
+ userData = ""
// Add all attributes into geometry user data
for (i <- 0 until geometryColId) userData += f.get(i) + "\t"
for (i <- geometryColId + 1 until f.size) userData += f.get(i) + "\t"
@@ -194,8 +196,9 @@ object Adapter {
dataFrame.rdd.map[Geometry](f => {
var geometry = f.get(0).asInstanceOf[Geometry]
var fieldSize = f.size
- var userData = ""
+ var userData:String = null
if (fieldSize > 1) {
+ userData = ""
// Add all attributes into geometry user data
for (i <- 1 until f.size) userData += f.get(i) + "\t"
userData = userData.dropRight(1)
diff --git a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
index aacd404..89dade2 100644
--- a/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
+++ b/sql/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/Functions.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.sedona_sql.expressions
import java.util
-import org.apache.sedona.core.geometryObjects.Circle
+import org.apache.sedona.core.geometryObjects.{Circle, GeoJSONWriterNew}
import org.apache.sedona.sql.utils.GeometrySerializer
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.InternalRow
@@ -42,7 +42,6 @@ import org.locationtech.jts.precision.GeometryPrecisionReducer
import org.locationtech.jts.simplify.TopologyPreservingSimplifier
import org.opengis.referencing.crs.CoordinateReferenceSystem
import org.opengis.referencing.operation.MathTransform
-import org.wololo.jts2geojson.GeoJSONWriter
import scala.collection.mutable.ArrayBuffer
import scala.util.{Failure, Success, Try}
@@ -496,7 +495,7 @@ case class ST_AsGeoJSON(inputExpressions: Seq[Expression])
inputExpressions.validateLength(1)
val geometry = inputExpressions.head.toGeometry(input)
- val writer = new GeoJSONWriter()
+ val writer = new GeoJSONWriterNew()
UTF8String.fromString(writer.write(geometry).toString)
}
diff --git a/sql/src/test/resources/babylon.linestring.properties b/sql/src/test/resources/babylon.linestring.properties
deleted file mode 100644
index f9b7b71..0000000
--- a/sql/src/test/resources/babylon.linestring.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=primaryroads-linestring.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/sql/src/test/resources/babylon.point.properties b/sql/src/test/resources/babylon.point.properties
deleted file mode 100644
index 3fb21a6..0000000
--- a/sql/src/test/resources/babylon.point.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=arealm.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/sql/src/test/resources/babylon.polygon.properties b/sql/src/test/resources/babylon.polygon.properties
deleted file mode 100644
index ef76396..0000000
--- a/sql/src/test/resources/babylon.polygon.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=primaryroads-polygon.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/sql/src/test/resources/babylon.rectangle.properties b/sql/src/test/resources/babylon.rectangle.properties
deleted file mode 100644
index 54a283d..0000000
--- a/sql/src/test/resources/babylon.rectangle.properties
+++ /dev/null
@@ -1,9 +0,0 @@
-inputLocation=zcta510.csv
-queryWindowSet=zcta510-small.csv
-offset=0
-splitter=csv
-gridType=rtree
-indexType=rtree
-numPartitions=5
-distance=0.01
-queryPolygonSet=primaryroads-polygon.csv
\ No newline at end of file
diff --git a/sql/src/test/resources/crs.test.properties b/sql/src/test/resources/crs.test.properties
index eebfcd3..a010c5c 100644
--- a/sql/src/test/resources/crs.test.properties
+++ b/sql/src/test/resources/crs.test.properties
@@ -1,7 +1,7 @@
inputLocation=crs-test-point.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.01
diff --git a/sql/src/test/resources/linestring.test.properties b/sql/src/test/resources/linestring.test.properties
index 1dca7e7..6b1d5ec 100644
--- a/sql/src/test/resources/linestring.test.properties
+++ b/sql/src/test/resources/linestring.test.properties
@@ -2,12 +2,12 @@ inputLocation=primaryroads-linestring.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=5
distance=0.01
queryPolygonSet=primaryroads-polygon.csv
-inputCount=13361
-inputBoundary=-158.104182, -65.648659, 17.982169, 49.002374
-matchCount=82227
-matchWithOriginalDuplicatesCount=128881
+inputCount=3000
+inputBoundary=-123.393766, -65.648659, 17.982169, 49.002374
+matchCount=535
+matchWithOriginalDuplicatesCount=875
diff --git a/sql/src/test/resources/point.test.properties b/sql/src/test/resources/point.test.properties
index c55aa5e..2a308a4 100644
--- a/sql/src/test/resources/point.test.properties
+++ b/sql/src/test/resources/point.test.properties
@@ -1,15 +1,15 @@
inputLocation=arealm-small.csv
queryWindowSet=zcta510-small.csv
-offset=0
+offset=1
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.01
queryPolygonSet=primaryroads-polygon.csv
-inputCount=20000
-inputBoundary=-179.147236, 179.475569, 26.992172, 71.355134
-rectangleMatchCount=8402
-rectangleMatchWithOriginalDuplicatesCount=8403
-polygonMatchCount=23569
-polygonMatchWithOriginalDuplicatesCount=25345
+inputCount=3000
+inputBoundary=-173.120769, -84.965961, 30.244859, 71.355134
+rectangleMatchCount=103
+rectangleMatchWithOriginalDuplicatesCount=103
+polygonMatchCount=472
+polygonMatchWithOriginalDuplicatesCount=562
diff --git a/sql/src/test/resources/polygon.test.properties b/sql/src/test/resources/polygon.test.properties
index deedabc..05526e1 100644
--- a/sql/src/test/resources/polygon.test.properties
+++ b/sql/src/test/resources/polygon.test.properties
@@ -2,15 +2,17 @@ inputLocation=primaryroads-polygon.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=5
distance=0.01
queryPolygonSet=primaryroads-polygon.csv
inputLocationGeojson=testPolygon.json
-inputCount=13361
-inputBoundary=-158.104182, -65.648659, 17.982169, 49.002374
-containsMatchCount=25795
-containsMatchWithOriginalDuplicatesCount=39729
-intersectsMatchCount=96411
-intersectsMatchWithOriginalDuplicatesCount=148381
+inputLocationWkt=county_small.tsv
+inputLocationWkb=county_small_wkb.tsv
+inputCount=3000
+inputBoundary=-158.104182, -66.03575, 17.986328, 48.645133
+containsMatchCount=6941
+containsMatchWithOriginalDuplicatesCount=9334
+intersectsMatchCount=24323
+intersectsMatchWithOriginalDuplicatesCount=32726
diff --git a/sql/src/test/resources/rectangle.test.properties b/sql/src/test/resources/rectangle.test.properties
index 6d16a95..26f3a24 100644
--- a/sql/src/test/resources/rectangle.test.properties
+++ b/sql/src/test/resources/rectangle.test.properties
@@ -2,12 +2,12 @@ inputLocation=zcta510-small.csv
queryWindowSet=zcta510-small.csv
offset=0
splitter=csv
-gridType=rtree
+gridType=kdbtree
indexType=rtree
numPartitions=11
distance=0.001
queryPolygonSet=primaryroads-polygon.csv
-inputCount=10000
-inputBoundary=-171.090042, 145.830505, -14.373765, 68.040264
-matchCount=71273
-matchWithOriginalDuplicatesCount=71412
+inputCount=3000
+inputBoundary=-171.090042, 145.830505, -14.373765, 49.00127
+matchCount=17599
+matchWithOriginalDuplicatesCount=17738
diff --git a/sql/src/test/resources/scalastyle_config.xml b/sql/src/test/resources/scalastyle_config.xml
index 6a95f66..02508b1 100644
--- a/sql/src/test/resources/scalastyle_config.xml
+++ b/sql/src/test/resources/scalastyle_config.xml
@@ -8,21 +8,24 @@
</check>
<check class="org.scalastyle.file.HeaderMatchesChecker" level="warning" enabled="true">
<parameters>
- <parameter name="header"><![CDATA[// Copyright (C) 2011-2012 the original author or authors.
-// See the LICENCE.txt file distributed with this work for additional
-// information regarding copyright ownership.
-//
-// 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
-//
-// 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.]]></parameter>
+ <parameter name="header"><![CDATA[/*
+ * 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.
+ */]]></parameter>
</parameters>
</check>
<check class="org.scalastyle.scalariform.SpacesAfterPlusChecker" level="warning" enabled="true"></check>
@@ -39,17 +42,17 @@
<parameter name="ignoreImports"><![CDATA[false]]></parameter>
</parameters>
</check>
- <check class="org.scalastyle.scalariform.ClassNamesChecker" level="warning" enabled="true">
+ <check class="org.scalastyle.scalariform.ClassNamesChecker" level="warning" enabled="false">
<parameters>
<parameter name="regex"><![CDATA[^[A-Z][A-Za-z]*$]]></parameter>
</parameters>
</check>
- <check class="org.scalastyle.scalariform.ObjectNamesChecker" level="warning" enabled="true">
+ <check class="org.scalastyle.scalariform.ObjectNamesChecker" level="warning" enabled="false">
<parameters>
<parameter name="regex"><![CDATA[^[A-Z][A-Za-z]*$]]></parameter>
</parameters>
</check>
- <check class="org.scalastyle.scalariform.PackageObjectNamesChecker" level="warning" enabled="true">
+ <check class="org.scalastyle.scalariform.PackageObjectNamesChecker" level="warning" enabled="false">
<parameters>
<parameter name="regex"><![CDATA[^[a-z][A-Za-z]*$]]></parameter>
</parameters>
@@ -65,7 +68,7 @@
<parameter name="maxParameters"><![CDATA[8]]></parameter>
</parameters>
</check>
- <check class="org.scalastyle.scalariform.MagicNumberChecker" level="warning" enabled="true">
+ <check class="org.scalastyle.scalariform.MagicNumberChecker" level="warning" enabled="false">
<parameters>
<parameter name="ignore"><![CDATA[-1,0,1,2,3]]></parameter>
</parameters>
@@ -73,7 +76,7 @@
<check class="org.scalastyle.scalariform.NoWhitespaceBeforeLeftBracketChecker" level="warning" enabled="true"></check>
<check class="org.scalastyle.scalariform.NoWhitespaceAfterLeftBracketChecker" level="warning" enabled="true"></check>
<check class="org.scalastyle.scalariform.ReturnChecker" level="warning" enabled="true"></check>
- <check class="org.scalastyle.scalariform.NullChecker" level="warning" enabled="true"></check>
+ <check class="org.scalastyle.scalariform.NullChecker" level="warning" enabled="false"></check>
<check class="org.scalastyle.scalariform.NoCloneChecker" level="warning" enabled="true"></check>
<check class="org.scalastyle.scalariform.NoFinalizeChecker" level="warning" enabled="true"></check>
<check class="org.scalastyle.scalariform.CovariantEqualsChecker" level="warning" enabled="true"></check>
@@ -96,7 +99,7 @@
</check>
<check class="org.scalastyle.scalariform.UppercaseLChecker" level="warning" enabled="true"></check>
<check class="org.scalastyle.scalariform.SimplifyBooleanExpressionChecker" level="warning" enabled="true"></check>
- <check class="org.scalastyle.scalariform.IfBraceChecker" level="warning" enabled="true">
+ <check class="org.scalastyle.scalariform.IfBraceChecker" level="warning" enabled="false">
<parameters>
<parameter name="singleLineAllowed"><![CDATA[true]]></parameter>
<parameter name="doubleLineAllowed"><![CDATA[false]]></parameter>
diff --git a/viz/src/main/java/org/apache/sedona/viz/showcase/Example.java b/viz/src/main/java/org/apache/sedona/viz/showcase/Example.java
index 9a6ede6..a228efb 100644
--- a/viz/src/main/java/org/apache/sedona/viz/showcase/Example.java
+++ b/viz/src/main/java/org/apache/sedona/viz/showcase/Example.java
@@ -265,8 +265,8 @@ public class Example
try {
PointRDD spatialRDD = new PointRDD(sparkContext, PointInputLocation, PointOffset, PointSplitter, false, PointNumPartitions, StorageLevel.MEMORY_ONLY());
PolygonRDD queryRDD = new PolygonRDD(sparkContext, PolygonInputLocation, PolygonSplitter, false, PolygonNumPartitions, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- queryRDD.spatialPartitioning(spatialRDD.grids);
+ spatialRDD.spatialPartitioning(GridType.KDBTREE);
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
spatialRDD.buildIndex(IndexType.RTREE, true);
JavaPairRDD<Polygon, Long> joinResult = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, true, false);
diff --git a/viz/src/main/scala/org/apache/sedona/viz/showcase/ScalaExample.scala b/viz/src/main/scala/org/apache/sedona/viz/showcase/ScalaExample.scala
index 178cf5c..3d3652f 100644
--- a/viz/src/main/scala/org/apache/sedona/viz/showcase/ScalaExample.scala
+++ b/viz/src/main/scala/org/apache/sedona/viz/showcase/ScalaExample.scala
@@ -142,8 +142,8 @@ object ScalaExample extends App {
def buildChoroplethMap(outputPath: String): Boolean = {
val spatialRDD = new PointRDD(sparkContext, PointInputLocation, PointOffset, PointSplitter, false, PointNumPartitions, StorageLevel.MEMORY_ONLY)
val queryRDD = new PolygonRDD(sparkContext, PolygonInputLocation, PolygonSplitter, false, PolygonNumPartitions, StorageLevel.MEMORY_ONLY)
- spatialRDD.spatialPartitioning(GridType.RTREE)
- queryRDD.spatialPartitioning(spatialRDD.grids)
+ spatialRDD.spatialPartitioning(GridType.KDBTREE)
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner)
spatialRDD.buildIndex(IndexType.RTREE, true)
val joinResult = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, true, false)
val visualizationOperator = new ChoroplethMap(1000, 600, USMainLandBoundary, false)
diff --git a/viz/src/test/java/org/apache/sedona/viz/ChoroplethmapTest.java b/viz/src/test/java/org/apache/sedona/viz/ChoroplethmapTest.java
index cb307f0..812d2fe 100644
--- a/viz/src/test/java/org/apache/sedona/viz/ChoroplethmapTest.java
+++ b/viz/src/test/java/org/apache/sedona/viz/ChoroplethmapTest.java
@@ -55,8 +55,8 @@ public class ChoroplethmapTest
{
PointRDD spatialRDD = new PointRDD(sparkContext, PointInputLocation, PointOffset, PointSplitter, false, PointNumPartitions, StorageLevel.MEMORY_ONLY());
RectangleRDD queryRDD = new RectangleRDD(sparkContext, RectangleInputLocation, RectangleSplitter, false, RectangleNumPartitions, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- queryRDD.spatialPartitioning(spatialRDD.grids);
+ spatialRDD.spatialPartitioning(GridType.KDBTREE);
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
spatialRDD.buildIndex(IndexType.RTREE, true);
JavaPairRDD<Polygon, Long> joinResult = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, true, true);
@@ -87,8 +87,8 @@ public class ChoroplethmapTest
//UserSuppliedPolygonMapper userSuppliedPolygonMapper = new UserSuppliedPolygonMapper();
PointRDD spatialRDD = new PointRDD(sparkContext, PointInputLocation, PointOffset, PointSplitter, false, PointNumPartitions, StorageLevel.MEMORY_ONLY());
PolygonRDD queryRDD = new PolygonRDD(sparkContext, PolygonInputLocation, PolygonSplitter, false, PolygonNumPartitions, StorageLevel.MEMORY_ONLY());
- spatialRDD.spatialPartitioning(GridType.RTREE);
- queryRDD.spatialPartitioning(spatialRDD.grids);
+ spatialRDD.spatialPartitioning(GridType.KDBTREE);
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner());
spatialRDD.buildIndex(IndexType.RTREE, true);
JavaPairRDD<Polygon, Long> joinResult = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, true, true);
diff --git a/viz/src/test/scala/org/apache/sedona/viz/rdd/scalaTest.scala b/viz/src/test/scala/org/apache/sedona/viz/rdd/scalaTest.scala
index 7ac0d3b..e176580 100644
--- a/viz/src/test/scala/org/apache/sedona/viz/rdd/scalaTest.scala
+++ b/viz/src/test/scala/org/apache/sedona/viz/rdd/scalaTest.scala
@@ -129,8 +129,8 @@ class scalaTest extends FunSpec with BeforeAndAfterAll{
it("should pass choropleth map") {
val spatialRDD = new PointRDD(sparkContext, PointInputLocation, PointOffset, PointSplitter, false, PointNumPartitions, StorageLevel.MEMORY_ONLY)
val queryRDD = new PolygonRDD(sparkContext, PolygonInputLocation, PolygonSplitter, false, PolygonNumPartitions, StorageLevel.MEMORY_ONLY)
- spatialRDD.spatialPartitioning(GridType.RTREE)
- queryRDD.spatialPartitioning(spatialRDD.grids)
+ spatialRDD.spatialPartitioning(GridType.KDBTREE)
+ queryRDD.spatialPartitioning(spatialRDD.getPartitioner)
spatialRDD.buildIndex(IndexType.RTREE, true)
val joinResult = JoinQuery.SpatialJoinQueryCountByKey(spatialRDD, queryRDD, true, false)
val visualizationOperator = new ChoroplethMap(1000, 600, USMainLandBoundary, false)