-
Notifications
You must be signed in to change notification settings - Fork 760
[GH-2939] Box2D spatial join: ST_BoxIntersects / ST_BoxContains #2953
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Draft
jiayuasu
wants to merge
2
commits into
apache:master
Choose a base branch
from
jiayuasu:feature/box2d-spatial-join
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Draft
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
187 changes: 187 additions & 0 deletions
187
spark/common/src/test/scala/org/apache/sedona/sql/Box2DJoinSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,187 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one | ||
| * or more contributor license agreements. See the NOTICE file | ||
| * distributed with this work for additional information | ||
| * regarding copyright ownership. The ASF licenses this file | ||
| * to you under the Apache License, Version 2.0 (the | ||
| * "License"); you may not use this file except in compliance | ||
| * with the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, | ||
| * software distributed under the License is distributed on an | ||
| * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY | ||
| * KIND, either express or implied. See the License for the | ||
| * specific language governing permissions and limitations | ||
| * under the License. | ||
| */ | ||
| package org.apache.sedona.sql | ||
|
|
||
| import org.apache.spark.sql.DataFrame | ||
| import org.apache.spark.sql.functions.{broadcast, expr} | ||
| import org.apache.spark.sql.sedona_sql.strategy.join.{BroadcastIndexJoinExec, RangeJoinExec} | ||
|
|
||
| class Box2DJoinSuite extends TestBaseScala { | ||
|
|
||
| import Box2DJoinSuite.TestBox | ||
|
|
||
| /** | ||
| * Three left-side boxes and three right-side boxes wired so we can predict exact result sizes: | ||
| * | ||
| * - L1=(0,0,10,10) R1=(5,5,15,15) — overlapping | ||
| * - L1=(0,0,10,10) R2=(2,2,8,8) — R2 fully inside L1 | ||
| * - L2=(0,0,10,10) R1=(5,5,15,15) — overlapping | ||
| * - L2=(0,0,10,10) R2=(2,2,8,8) — R2 fully inside L2 | ||
| * - L3 and R3 are disjoint from everything else; (L3,R3) is itself disjoint. | ||
| * | ||
| * Intersection-pair count: 4. Containment-pair count: 2 (L1⊇R2, L2⊇R2). | ||
| */ | ||
| private def leftBoxes: DataFrame = { | ||
| import sparkSession.implicits._ | ||
| Seq(TestBox(1, 0, 0, 10, 10), TestBox(2, 0, 0, 10, 10), TestBox(3, 20, 20, 30, 30)) | ||
| .toDF("id", "xmin", "ymin", "xmax", "ymax") | ||
| .selectExpr("id", "ST_MakeBox2D(ST_Point(xmin, ymin), ST_Point(xmax, ymax)) AS box") | ||
| } | ||
|
|
||
| private def rightBoxes: DataFrame = { | ||
| import sparkSession.implicits._ | ||
| Seq(TestBox(11, 5, 5, 15, 15), TestBox(12, 2, 2, 8, 8), TestBox(13, 40, 40, 50, 50)) | ||
| .toDF("id", "xmin", "ymin", "xmax", "ymax") | ||
| .selectExpr("id", "ST_MakeBox2D(ST_Point(xmin, ymin), ST_Point(xmax, ymax)) AS box") | ||
| } | ||
|
|
||
| describe("Box2D spatial join") { | ||
|
|
||
| it("ST_BoxIntersects: broadcast index join produces correct pairs") { | ||
| val df = leftBoxes | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| val plan = df.queryExecution.sparkPlan | ||
|
Comment on lines
+54
to
+60
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Added in 9144fb8: 'Null Box2D rows are safe and produce no matches' (covers both broadcast and range paths) and 'Inverted Box2D bounds in a join throw IllegalArgumentException' (verifies the exception type and message). |
||
| assert( | ||
| plan.collect { case b: BroadcastIndexJoinExec => b }.size == 1, | ||
| "Expected BroadcastIndexJoinExec in the plan") | ||
| assert(df.count() == 4) | ||
| } | ||
|
|
||
| it("ST_BoxIntersects: argument order is symmetric") { | ||
| val swapped = leftBoxes | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxIntersects(R.box, L.box)")) | ||
| assert(swapped.count() == 4) | ||
| assert(swapped.queryExecution.sparkPlan.collect { case b: BroadcastIndexJoinExec => | ||
| b | ||
| }.size == 1) | ||
| } | ||
|
|
||
| it("ST_BoxContains: broadcast index join uses COVERS semantics") { | ||
| val df = leftBoxes | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxContains(L.box, R.box)")) | ||
| assert(df.queryExecution.sparkPlan.collect { case b: BroadcastIndexJoinExec => | ||
| b | ||
| }.size == 1) | ||
| assert(df.count() == 2) | ||
| } | ||
|
|
||
| it("ST_BoxContains: edge-touching boxes count (closed-interval semantics)") { | ||
| // R contained in L sharing an edge: ST_BoxContains is closed-interval, so this matches. | ||
| // JTS Polygon.contains would reject (strict-interior), JTS Polygon.covers accepts; the | ||
| // detector maps ST_BoxContains → SpatialPredicate.COVERS specifically for this case. | ||
| import sparkSession.implicits._ | ||
| val outer = Seq(TestBox(1, 0, 0, 10, 10)) | ||
| .toDF("id", "xmin", "ymin", "xmax", "ymax") | ||
| .selectExpr("id", "ST_MakeBox2D(ST_Point(xmin, ymin), ST_Point(xmax, ymax)) AS box") | ||
| // edge-sharing box: same xmax, shares the right edge with outer. | ||
| val inner = Seq(TestBox(11, 5, 5, 10, 10)) | ||
| .toDF("id", "xmin", "ymin", "xmax", "ymax") | ||
| .selectExpr("id", "ST_MakeBox2D(ST_Point(xmin, ymin), ST_Point(xmax, ymax)) AS box") | ||
| val df = outer | ||
| .alias("O") | ||
| .join(broadcast(inner.alias("I")), expr("ST_BoxContains(O.box, I.box)")) | ||
| assert(df.count() == 1, "Closed-interval containment must include edge-touching boxes") | ||
| } | ||
|
|
||
| it("ST_BoxIntersects: non-broadcast range join produces the same count") { | ||
| val df = leftBoxes | ||
| .alias("L") | ||
| .join(rightBoxes.alias("R"), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| assert( | ||
| df.queryExecution.sparkPlan.collect { case r: RangeJoinExec => r }.size == 1, | ||
| "Expected RangeJoinExec in the plan") | ||
| assert(df.count() == 4) | ||
| } | ||
|
|
||
| it("Null Box2D rows are safe and produce no matches") { | ||
| // A null shape on either side must not crash the executor and must not contribute matches | ||
| // (mirrors the existing GeometrySerializer.deserialize(null) → empty-collection fallback). | ||
| import sparkSession.implicits._ | ||
| val withNullLeft = leftBoxes | ||
| .selectExpr("id", "box AS box") | ||
| .union(Seq((99, null.asInstanceOf[org.apache.sedona.common.geometryObjects.Box2D])) | ||
| .toDF("id", "box")) | ||
| val df = withNullLeft | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| assert(df.count() == 4) // unchanged from the non-null fixture | ||
| // Range join path (no broadcast) also tolerates nulls. | ||
| val rangeDf = withNullLeft | ||
| .alias("L") | ||
| .join(rightBoxes.alias("R"), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| assert(rangeDf.count() == 4) | ||
| } | ||
|
|
||
| it("Inverted Box2D bounds in a join throw IllegalArgumentException") { | ||
| import sparkSession.implicits._ | ||
| // Construct an inverted Box2D directly via the Java constructor (the SQL ST_MakeBox2D | ||
| // doesn't validate, so this is how a stored column with inverted bounds would look). | ||
| val invertedLeft = | ||
| Seq((1, new org.apache.sedona.common.geometryObjects.Box2D(10.0, 0.0, 0.0, 10.0))) | ||
| .toDF("id", "box") | ||
| val ex = intercept[org.apache.spark.SparkException] { | ||
| invertedLeft | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| .collect() | ||
|
Comment on lines
+141
to
+145
|
||
| } | ||
| val cause = Iterator | ||
| .iterate(ex: Throwable)(_.getCause) | ||
| .takeWhile(_ != null) | ||
| .find(_.isInstanceOf[IllegalArgumentException]) | ||
| assert(cause.isDefined, s"Expected IllegalArgumentException in cause chain, got: $ex") | ||
| assert(cause.get.getMessage.contains("inverted bounds")) | ||
| } | ||
|
|
||
| it("Result is equivalent to ST_Intersects on the Box2D-as-polygon envelopes") { | ||
| val viaBox = leftBoxes | ||
| .alias("L") | ||
| .join(broadcast(rightBoxes.alias("R")), expr("ST_BoxIntersects(L.box, R.box)")) | ||
| .selectExpr("L.id AS l", "R.id AS r") | ||
| .orderBy("l", "r") | ||
| .collect() | ||
| .toSeq | ||
|
|
||
| // ST_GeomFromBox2D is the function-form equivalent of `CAST(box AS geometry)`. The cast | ||
| // syntax requires the Sedona SQL parser extension; this suite runs under the common test | ||
| // base, which doesn't wire that extension, so we go through the function form here. | ||
| val asPolygons = leftBoxes | ||
| .selectExpr("id", "ST_GeomFromBox2D(box) AS g") | ||
| .alias("L") | ||
| .join( | ||
| broadcast(rightBoxes.selectExpr("id", "ST_GeomFromBox2D(box) AS g").alias("R")), | ||
| expr("ST_Intersects(L.g, R.g)")) | ||
| .selectExpr("L.id AS l", "R.id AS r") | ||
| .orderBy("l", "r") | ||
| .collect() | ||
| .toSeq | ||
|
|
||
| assert(viaBox == asPolygons) | ||
| } | ||
| } | ||
|
|
||
| } | ||
|
|
||
| object Box2DJoinSuite { | ||
| // Top-level case class so Spark's encoder doesn't need an outer-class reference. | ||
| case class TestBox(id: Int, xmin: Double, ymin: Double, xmax: Double, ymax: Double) | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed in 9144fb8 — the helper now keeps returning null on null inputs, and toSpatialRDD/toExpandedEnvelopeRDD wrap it in a shapeToGeometryOrEmpty that substitutes an empty GeometryCollection. This matches GeometrySerializer.deserialize(null)'s legacy behaviour. The KnnJoinSuite null-geom regression is gone in local runs.