jiayuasu commented on code in PR #2953:
URL: https://github.com/apache/sedona/pull/2953#discussion_r3250494928


##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryBase.scala:
##########
@@ -49,8 +51,7 @@ trait TraitJoinQueryBase {
     spatialRdd.setRawSpatialRDD(
       rdd
         .map { x =>
-          val shape =
-            
GeometrySerializer.deserialize(shapeExpression.eval(x).asInstanceOf[Array[Byte]])
+          val shape = TraitJoinQueryBase.shapeToGeometry(shapeExpression, x)
           shape.setUserData(x.copy)
           shape

Review Comment:
   Fixed in 9144fb82 — 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.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/TraitJoinQueryBase.scala:
##########
@@ -178,3 +178,36 @@ trait TraitJoinQueryBase {
     }
   }
 }
+
+object TraitJoinQueryBase {
+
+  /**
+   * Materialise a shape column value as a JTS [[Geometry]]. Box2D-typed 
columns are turned into
+   * the closed rectangular polygon implied by their `(xmin, ymin, xmax, 
ymax)` bounds; all other
+   * shape columns are deserialised from the Sedona geometry binary form.
+   *
+   * Producing a JTS rectangle here lets the rest of the join machinery — 
partitioner, R-tree
+   * `IndexBuilder`, refine evaluator — stay shape-agnostic. JTS already 
short-circuits
+   * rectangle-rectangle predicates (`Polygon.isRectangle` triggers 
`RectangleIntersects` /
+   * `RectangleContains`), so a `ST_BoxIntersects` join naturally pays only 
the four-double
+   * envelope comparison at refine time.
+   */
+  def shapeToGeometry(shapeExpression: Expression, row: InternalRow): Geometry 
= {
+    val evaluated = shapeExpression.eval(row)
+    if (evaluated == null) {
+      null
+    } else
+      shapeExpression.dataType match {
+        case _: Box2DUDT =>
+          val box = evaluated.asInstanceOf[InternalRow]
+          Constructors
+            .polygonFromEnvelope(
+              box.getDouble(0),
+              box.getDouble(1),
+              box.getDouble(2),
+              box.getDouble(3))

Review Comment:
   Fixed in 9144fb82 — the Box2D branch now validates xmin <= xmax and ymin <= 
ymax and throws the same IllegalArgumentException raised by 
Predicates.boxIntersects/boxContains. Kept polygonFromEnvelope as the 
materialiser; JTS handles degenerate boxes (zero-area polygon) correctly in 
both index and refine paths, and switching to geomFromBox2D would introduce 
shape polymorphism (Point/Line/Polygon) at the join boundary for what's 
nominally a single rectangle type.



##########
spark/common/src/test/scala/org/apache/sedona/sql/Box2DJoinSuite.scala:
##########
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.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

Review Comment:
   Added in 9144fb82: '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).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to