Copilot commented on code in PR #2831:
URL: https://github.com/apache/sedona/pull/2831#discussion_r3104397750


##########
common/src/main/java/org/apache/sedona/common/S2Geography/WKBGeography.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.common.S2Geography;
+
+import com.google.common.geometry.S2CellId;
+import com.google.common.geometry.S2LatLng;
+import com.google.common.geometry.S2Point;
+import com.google.common.geometry.S2PointRegion;
+import com.google.common.geometry.S2Region;
+import com.google.common.geometry.S2Shape;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.List;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.PrecisionModel;
+import org.locationtech.jts.io.ParseException;
+
+/**
+ * A Geography implementation that stores WKB bytes as the primary 
representation, with lazy-parsed
+ * JTS Geometry and S2 Geography caches. This enables zero-parse construction 
from WKB and deferred
+ * S2 parsing only when spherical operations are needed.
+ *
+ * <p>Key optimizations (per paleolimbot's review): - dimension() reads WKB 
type byte directly (no
+ * S2 parse) - region()/getCellUnionBound() for points read coordinates from 
WKB (no S2 parse) -
+ * shape()/numShapes() use WkbS2Shape for simple types (no S2Polygon 
construction) - ShapeIndex is
+ * built from WkbS2Shape directly (skips Geography layer)
+ */
+public class WKBGeography extends Geography {
+
+  /**
+   * When true, fromWKB() eagerly builds the S2 Geography and ShapeIndex at 
construction time
+   * instead of lazily on first access. This eliminates cold-path overhead for 
predicate-heavy
+   * workloads (ST_Contains, ST_Intersects) at the cost of slower 
deserialization for metric-only
+   * workloads. Set via spark.sedona.geography.eagerShapeIndex or 
setEagerShapeIndex(). Default
+   * false.
+   */
+  private static volatile boolean eagerShapeIndex = false;
+
+  public static void setEagerShapeIndex(boolean eager) {
+    eagerShapeIndex = eager;
+  }
+
+  public static boolean isEagerShapeIndex() {
+    return eagerShapeIndex;
+  }
+
+  private final byte[] wkbBytes;
+
+  // Lazy caches — volatile for thread safety with double-checked locking
+  private volatile Geometry jtsGeometry;
+  private volatile Geography s2Geography;
+  private volatile ShapeIndexGeography shapeIndexGeography;
+
+  private WKBGeography(byte[] wkbBytes, int srid) {
+    super(GeographyKind.UNINITIALIZED);
+    this.wkbBytes = wkbBytes;
+    setSRID(srid);
+  }
+
+  /**
+   * Create a WKBGeography from raw WKB bytes. When eagerShapeIndex is false 
(default), this is
+   * zero-parse — just wraps the byte array. When eager mode is enabled, this 
also builds the
+   * ShapeIndex upfront.
+   */
+  public static WKBGeography fromWKB(byte[] wkb, int srid) {
+    WKBGeography geog = new WKBGeography(wkb, srid);
+    if (eagerShapeIndex) {
+      geog.getShapeIndexGeography();
+    }
+    return geog;
+  }
+
+  /** Create a WKBGeography from a JTS Geometry by serializing it to WKB. */
+  public static WKBGeography fromJTS(Geometry jts) {
+    org.locationtech.jts.io.WKBWriter writer =
+        new org.locationtech.jts.io.WKBWriter(
+            2, org.locationtech.jts.io.ByteOrderValues.LITTLE_ENDIAN);
+    byte[] wkb = writer.write(jts);
+    WKBGeography geog = new WKBGeography(wkb, jts.getSRID());
+    geog.jtsGeometry = jts;
+    return geog;
+  }
+
+  /** Create a WKBGeography from an existing S2 Geography by converting it to 
WKB. */
+  public static WKBGeography fromS2Geography(Geography s2geog) {
+    WKBWriter writer =
+        new WKBWriter(2, 
org.locationtech.jts.io.ByteOrderValues.LITTLE_ENDIAN, false);
+    byte[] wkb = writer.write(s2geog);
+    WKBGeography geog = new WKBGeography(wkb, s2geog.getSRID());
+    geog.s2Geography = s2geog;
+    return geog;
+  }
+
+  /** Returns the raw WKB bytes. Zero cost. */
+  public byte[] getWKBBytes() {
+    return wkbBytes;
+  }
+
+  /** Returns a JTS Geometry, lazily parsed from WKB on first access. */
+  public Geometry getJTSGeometry() {
+    Geometry result = jtsGeometry;
+    if (result == null) {
+      synchronized (this) {
+        result = jtsGeometry;
+        if (result == null) {
+          try {
+            org.locationtech.jts.io.WKBReader reader = new 
org.locationtech.jts.io.WKBReader();
+            result = reader.read(wkbBytes);
+            result.setSRID(getSRID());
+          } catch (ParseException e) {
+            throw new RuntimeException("Failed to parse WKB to JTS Geometry", 
e);
+          }
+          jtsGeometry = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Returns a ShapeIndexGeography, lazily built on first access. For simple 
types (Point,
+   * LineString, Polygon), builds the index directly from WkbS2Shape — no S2 
Geography construction
+   * needed.
+   */
+  public ShapeIndexGeography getShapeIndexGeography() {
+    ShapeIndexGeography result = shapeIndexGeography;
+    if (result == null) {
+      synchronized (this) {
+        result = shapeIndexGeography;
+        if (result == null) {
+          int type = wkbBaseType();
+          if (type >= 1 && type <= 3) {
+            // Point/LineString/Polygon: build ShapeIndex from WkbS2Shape
+            // Avoids S2Loop/S2Polygon internal index builds
+            result = new ShapeIndexGeography();
+            result.shapeIndex.add(new WkbS2Shape(wkbBytes));
+          } else {
+            // Multi-types and collections fall back to full S2 parse
+            result = new ShapeIndexGeography(getS2Geography());
+          }
+          shapeIndexGeography = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  /** Returns an S2 Geography, lazily parsed from WKB on first access. */
+  public Geography getS2Geography() {
+    Geography result = s2Geography;
+    if (result == null) {
+      synchronized (this) {
+        result = s2Geography;
+        if (result == null) {
+          try {
+            WKBReader reader = new WKBReader();
+            result = reader.read(wkbBytes);
+            result.setSRID(getSRID());
+          } catch (ParseException e) {
+            throw new RuntimeException("Failed to parse WKB to S2 Geography", 
e);
+          }
+          s2Geography = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  // ─── WKB-direct optimizations (no S2 parse needed) ─────────────────────
+
+  /** Returns the base WKB geometry type (1-7), masking off Z/M/SRID flags. */
+  private int wkbBaseType() {
+    boolean le = (wkbBytes[0] == 0x01);
+    int raw;
+    if (le) {
+      raw =
+          (wkbBytes[1] & 0xFF)
+              | ((wkbBytes[2] & 0xFF) << 8)
+              | ((wkbBytes[3] & 0xFF) << 16)
+              | ((wkbBytes[4] & 0xFF) << 24);
+    } else {
+      raw =
+          ((wkbBytes[1] & 0xFF) << 24)
+              | ((wkbBytes[2] & 0xFF) << 16)
+              | ((wkbBytes[3] & 0xFF) << 8)
+              | (wkbBytes[4] & 0xFF);
+    }
+    return raw & 0xFF;
+  }
+
+  /** Returns true if this WKB represents a single Point (type 1). */
+  public boolean isPoint() {
+    return wkbBaseType() == 1;
+  }
+
+  /** Extract the S2Point from a Point WKB without full S2 parse. */
+  public S2Point extractPoint() {
+    boolean le = (wkbBytes[0] == 0x01);
+    ByteBuffer bb =
+        ByteBuffer.wrap(wkbBytes).order(le ? ByteOrder.LITTLE_ENDIAN : 
ByteOrder.BIG_ENDIAN);
+    double lon = bb.getDouble(5);
+    double lat = bb.getDouble(13);
+    return S2LatLng.fromDegrees(lat, lon).toPoint();

Review Comment:
   `extractPoint()` reads coordinates at fixed offsets (5/13), which is only 
correct for 2D WKB without an embedded SRID. For EWKB where the SRID flag is 
set, bytes 5–8 are the SRID and coordinates start at offset 9; this will 
produce wrong points and therefore wrong `region()`, `getCellUnionBound()`, and 
the point fast paths in geography `distance()`/predicates. Please derive the 
coordinate offset from the decoded `typeInt` (SRID/Z/M flags) before reading 
X/Y.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/strategy/join/JoinQueryDetector.scala:
##########
@@ -209,6 +199,16 @@ class JoinQueryDetector(sparkSession: SparkSession) 
extends SparkStrategy {
       val queryDetection: Option[JoinQueryDetection] = condition.flatMap {
         case joinConditionMatcher(predicate, extraCondition) =>
           predicate match {
+            case ST_Contains(Seq(leftShape, rightShape)) =>
+              Some(
+                JoinQueryDetection(
+                  left,
+                  right,
+                  leftShape,
+                  rightShape,
+                  SpatialPredicate.CONTAINS,
+                  false,
+                  extraCondition))

Review Comment:
   `ST_Contains` now supports Geography via `InferredExpression`, but this join 
planning path always treats it as a geometry join (`isGeography = false`) and 
the join execution path deserializes join keys with `GeometrySerializer` (see 
`TraitJoinQueryBase.toSpatialRDD`). If a user writes a join condition 
`ST_Contains(geogA, geogB)`, this optimization will likely throw at runtime by 
attempting to deserialize Geography bytes as Geometry. Since geography join 
support is explicitly deferred, this detector should skip planning spatial 
joins when either side is `GeographyUDT` (or otherwise detect geography inputs 
and fall back to Spark’s default join).



##########
common/src/main/java/org/apache/sedona/common/S2Geography/WkbS2Shape.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.common.S2Geography;
+
+import com.google.common.geometry.S2;
+import com.google.common.geometry.S2EdgeUtil;
+import com.google.common.geometry.S2LatLng;
+import com.google.common.geometry.S2Point;
+import com.google.common.geometry.S2Predicates;
+import com.google.common.geometry.S2Shape;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * An S2Shape implementation that reads WKB bytes once, converts all 
coordinates to S2Points in the
+ * constructor, and stores them in an array. This avoids constructing 
S2Loop/S2Polygon objects
+ * (which each build their own internal S2ShapeIndex), while also avoiding 
repeated trig calls on
+ * every getEdge() access.
+ *
+ * <p>Supports Point (type 1), LineString (type 2), and Polygon (type 3). 
Multi-types and
+ * collections should fall back to the full S2 Geography parse path.
+ */
+public class WkbS2Shape implements S2Shape {
+
+  private final int dim; // S2 dimension: 0=point, 1=line, 2=polygon
+  private final S2Point[] vertices; // all vertices, pre-converted from WKB
+  private final int totalEdges;
+  private final int[] chainStarts; // edge offset for each chain
+  private final int[] chainLengths; // edge count for each chain
+  private final int[] vertexOffsets; // index into vertices[] for first vertex 
of each chain
+
+  // For polygon containsOrigin — computed eagerly at construction for polygons
+  private final boolean containsOriginValue;
+
+  public WkbS2Shape(byte[] wkb) {
+    boolean le = (wkb[0] == 0x01);
+    ByteBuffer buf =
+        ByteBuffer.wrap(wkb).order(le ? ByteOrder.LITTLE_ENDIAN : 
ByteOrder.BIG_ENDIAN);
+    int wkbType = buf.getInt(1) & 0xFF;
+

Review Comment:
   WKB type decoding is incorrect here: `buf.getInt(1) & 0xFF` drops EWKB/ISO 
type information (e.g., Z/M types like 1001/3001) and can misclassify 
geometries. It should read the full `typeInt`, derive `geometryType = (typeInt 
& 0xffff) % 1000` (as `WKBReader` does), and also account for EWKB flags such 
as SRID (0x20000000) when computing the payload offset.



##########
spark/common/src/main/scala/org/apache/spark/sql/sedona_sql/expressions/FunctionResolver.scala:
##########
@@ -82,14 +82,10 @@ object FunctionResolver {
         if (ambiguousMatches.length == 1) {
           function
         } else {
-          // Detected ambiguous matches, throw exception
-          val candidateTypesMsg = ambiguousMatches
-            .map { case (function, _) =>
-              "  (" + function.sparkInputTypes.mkString(", ") + ")"
-            }
-            .mkString("\n")
-          throw new IllegalArgumentException(
-            "Ambiguous function call. Candidates are: \n" + candidateTypesMsg)
+          // Multiple candidates match equally (e.g., null inputs matching 
both Geometry and
+          // Geography overloads). Prefer the first candidate — for null 
inputs all overloads
+          // return null, so the choice is irrelevant.
+          ambiguousMatches.head._1
         }

Review Comment:
   The ambiguity handling change (`ambiguousMatches.head`) applies to *all* 
equally-good matches, not just the null-literal case mentioned in the comment. 
This can silently pick an arbitrary overload in situations where two overloads 
require the same number of implicit casts, changing behavior from a clear error 
to potentially wrong results. Safer options are: (1) keep throwing on ambiguity 
unless all ambiguous calls contain `NullType` arguments, or (2) add a 
deterministic tie-breaker based on a specificity/precedence rule that can’t 
change semantics unexpectedly.



##########
common/src/main/java/org/apache/sedona/common/S2Geography/WkbS2Shape.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.common.S2Geography;
+
+import com.google.common.geometry.S2;
+import com.google.common.geometry.S2EdgeUtil;
+import com.google.common.geometry.S2LatLng;
+import com.google.common.geometry.S2Point;
+import com.google.common.geometry.S2Predicates;
+import com.google.common.geometry.S2Shape;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+/**
+ * An S2Shape implementation that reads WKB bytes once, converts all 
coordinates to S2Points in the
+ * constructor, and stores them in an array. This avoids constructing 
S2Loop/S2Polygon objects
+ * (which each build their own internal S2ShapeIndex), while also avoiding 
repeated trig calls on
+ * every getEdge() access.
+ *
+ * <p>Supports Point (type 1), LineString (type 2), and Polygon (type 3). 
Multi-types and
+ * collections should fall back to the full S2 Geography parse path.
+ */
+public class WkbS2Shape implements S2Shape {
+
+  private final int dim; // S2 dimension: 0=point, 1=line, 2=polygon
+  private final S2Point[] vertices; // all vertices, pre-converted from WKB
+  private final int totalEdges;
+  private final int[] chainStarts; // edge offset for each chain
+  private final int[] chainLengths; // edge count for each chain
+  private final int[] vertexOffsets; // index into vertices[] for first vertex 
of each chain
+
+  // For polygon containsOrigin — computed eagerly at construction for polygons
+  private final boolean containsOriginValue;
+
+  public WkbS2Shape(byte[] wkb) {
+    boolean le = (wkb[0] == 0x01);
+    ByteBuffer buf =
+        ByteBuffer.wrap(wkb).order(le ? ByteOrder.LITTLE_ENDIAN : 
ByteOrder.BIG_ENDIAN);
+    int wkbType = buf.getInt(1) & 0xFF;
+
+    switch (wkbType) {
+      case 1: // Point
+        {
+          this.dim = 0;
+          double lon = buf.getDouble(5);
+          double lat = buf.getDouble(13);
+          S2Point p = S2LatLng.fromDegrees(lat, lon).toPoint();
+          this.vertices = new S2Point[] {p};
+          this.totalEdges = 1;
+          this.chainStarts = new int[] {0};
+          this.chainLengths = new int[] {1};
+          this.vertexOffsets = new int[] {0};
+          this.containsOriginValue = false;
+          break;
+        }
+
+      case 2: // LineString
+        {
+          this.dim = 1;
+          int numCoords = buf.getInt(5);
+          this.vertices = readVertices(buf, 9, numCoords);
+          this.totalEdges = Math.max(0, numCoords - 1);

Review Comment:
   The byte offsets here assume 2D WKB without an embedded SRID (e.g., Point 
X/Y at 5/13, LineString count at 5, Polygon ring count at 5). If the input is 
EWKB with the SRID flag set, coordinates start after the 4-byte SRID; if it’s 
Z/M, the coordinate stride is larger than 16 bytes. This will build an 
incorrect S2Shape (and therefore incorrect predicates/distances) for those 
valid inputs. Please compute a `coordOffset` based on the decoded EWKB flags 
and a stride based on input dimension, similar to `common/.../WKBReader.java`.



##########
common/src/main/java/org/apache/sedona/common/S2Geography/WKBGeography.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.common.S2Geography;
+
+import com.google.common.geometry.S2CellId;
+import com.google.common.geometry.S2LatLng;
+import com.google.common.geometry.S2Point;
+import com.google.common.geometry.S2PointRegion;
+import com.google.common.geometry.S2Region;
+import com.google.common.geometry.S2Shape;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.List;
+import org.locationtech.jts.geom.Geometry;
+import org.locationtech.jts.geom.PrecisionModel;
+import org.locationtech.jts.io.ParseException;
+
+/**
+ * A Geography implementation that stores WKB bytes as the primary 
representation, with lazy-parsed
+ * JTS Geometry and S2 Geography caches. This enables zero-parse construction 
from WKB and deferred
+ * S2 parsing only when spherical operations are needed.
+ *
+ * <p>Key optimizations (per paleolimbot's review): - dimension() reads WKB 
type byte directly (no
+ * S2 parse) - region()/getCellUnionBound() for points read coordinates from 
WKB (no S2 parse) -
+ * shape()/numShapes() use WkbS2Shape for simple types (no S2Polygon 
construction) - ShapeIndex is
+ * built from WkbS2Shape directly (skips Geography layer)
+ */
+public class WKBGeography extends Geography {
+
+  /**
+   * When true, fromWKB() eagerly builds the S2 Geography and ShapeIndex at 
construction time
+   * instead of lazily on first access. This eliminates cold-path overhead for 
predicate-heavy
+   * workloads (ST_Contains, ST_Intersects) at the cost of slower 
deserialization for metric-only
+   * workloads. Set via spark.sedona.geography.eagerShapeIndex or 
setEagerShapeIndex(). Default
+   * false.
+   */
+  private static volatile boolean eagerShapeIndex = false;
+
+  public static void setEagerShapeIndex(boolean eager) {
+    eagerShapeIndex = eager;
+  }
+
+  public static boolean isEagerShapeIndex() {
+    return eagerShapeIndex;
+  }
+
+  private final byte[] wkbBytes;
+
+  // Lazy caches — volatile for thread safety with double-checked locking
+  private volatile Geometry jtsGeometry;
+  private volatile Geography s2Geography;
+  private volatile ShapeIndexGeography shapeIndexGeography;
+
+  private WKBGeography(byte[] wkbBytes, int srid) {
+    super(GeographyKind.UNINITIALIZED);
+    this.wkbBytes = wkbBytes;
+    setSRID(srid);
+  }
+
+  /**
+   * Create a WKBGeography from raw WKB bytes. When eagerShapeIndex is false 
(default), this is
+   * zero-parse — just wraps the byte array. When eager mode is enabled, this 
also builds the
+   * ShapeIndex upfront.
+   */
+  public static WKBGeography fromWKB(byte[] wkb, int srid) {
+    WKBGeography geog = new WKBGeography(wkb, srid);
+    if (eagerShapeIndex) {
+      geog.getShapeIndexGeography();
+    }
+    return geog;
+  }
+
+  /** Create a WKBGeography from a JTS Geometry by serializing it to WKB. */
+  public static WKBGeography fromJTS(Geometry jts) {
+    org.locationtech.jts.io.WKBWriter writer =
+        new org.locationtech.jts.io.WKBWriter(
+            2, org.locationtech.jts.io.ByteOrderValues.LITTLE_ENDIAN);
+    byte[] wkb = writer.write(jts);
+    WKBGeography geog = new WKBGeography(wkb, jts.getSRID());
+    geog.jtsGeometry = jts;
+    return geog;
+  }
+
+  /** Create a WKBGeography from an existing S2 Geography by converting it to 
WKB. */
+  public static WKBGeography fromS2Geography(Geography s2geog) {
+    WKBWriter writer =
+        new WKBWriter(2, 
org.locationtech.jts.io.ByteOrderValues.LITTLE_ENDIAN, false);
+    byte[] wkb = writer.write(s2geog);
+    WKBGeography geog = new WKBGeography(wkb, s2geog.getSRID());
+    geog.s2Geography = s2geog;
+    return geog;
+  }
+
+  /** Returns the raw WKB bytes. Zero cost. */
+  public byte[] getWKBBytes() {
+    return wkbBytes;
+  }
+
+  /** Returns a JTS Geometry, lazily parsed from WKB on first access. */
+  public Geometry getJTSGeometry() {
+    Geometry result = jtsGeometry;
+    if (result == null) {
+      synchronized (this) {
+        result = jtsGeometry;
+        if (result == null) {
+          try {
+            org.locationtech.jts.io.WKBReader reader = new 
org.locationtech.jts.io.WKBReader();
+            result = reader.read(wkbBytes);
+            result.setSRID(getSRID());
+          } catch (ParseException e) {
+            throw new RuntimeException("Failed to parse WKB to JTS Geometry", 
e);
+          }
+          jtsGeometry = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Returns a ShapeIndexGeography, lazily built on first access. For simple 
types (Point,
+   * LineString, Polygon), builds the index directly from WkbS2Shape — no S2 
Geography construction
+   * needed.
+   */
+  public ShapeIndexGeography getShapeIndexGeography() {
+    ShapeIndexGeography result = shapeIndexGeography;
+    if (result == null) {
+      synchronized (this) {
+        result = shapeIndexGeography;
+        if (result == null) {
+          int type = wkbBaseType();
+          if (type >= 1 && type <= 3) {
+            // Point/LineString/Polygon: build ShapeIndex from WkbS2Shape
+            // Avoids S2Loop/S2Polygon internal index builds
+            result = new ShapeIndexGeography();
+            result.shapeIndex.add(new WkbS2Shape(wkbBytes));
+          } else {
+            // Multi-types and collections fall back to full S2 parse
+            result = new ShapeIndexGeography(getS2Geography());
+          }
+          shapeIndexGeography = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  /** Returns an S2 Geography, lazily parsed from WKB on first access. */
+  public Geography getS2Geography() {
+    Geography result = s2Geography;
+    if (result == null) {
+      synchronized (this) {
+        result = s2Geography;
+        if (result == null) {
+          try {
+            WKBReader reader = new WKBReader();
+            result = reader.read(wkbBytes);
+            result.setSRID(getSRID());
+          } catch (ParseException e) {
+            throw new RuntimeException("Failed to parse WKB to S2 Geography", 
e);
+          }
+          s2Geography = result;
+        }
+      }
+    }
+    return result;
+  }
+
+  // ─── WKB-direct optimizations (no S2 parse needed) ─────────────────────
+
+  /** Returns the base WKB geometry type (1-7), masking off Z/M/SRID flags. */
+  private int wkbBaseType() {
+    boolean le = (wkbBytes[0] == 0x01);
+    int raw;
+    if (le) {
+      raw =
+          (wkbBytes[1] & 0xFF)
+              | ((wkbBytes[2] & 0xFF) << 8)
+              | ((wkbBytes[3] & 0xFF) << 16)
+              | ((wkbBytes[4] & 0xFF) << 24);
+    } else {
+      raw =
+          ((wkbBytes[1] & 0xFF) << 24)
+              | ((wkbBytes[2] & 0xFF) << 16)
+              | ((wkbBytes[3] & 0xFF) << 8)
+              | (wkbBytes[4] & 0xFF);
+    }
+    return raw & 0xFF;
+  }

Review Comment:
   `wkbBaseType()` claims to mask off Z/M/SRID flags, but `raw & 0xFF` only 
keeps the lowest byte. This breaks ISO WKB Z/M encodings (e.g., 1001/3001) and 
doesn’t actually remove the EWKB SRID flag safely. Consider decoding `typeInt` 
the same way as `WKBReader` (e.g., `geometryType = (typeInt & 0xffff) % 1000`) 
and separately tracking whether SRID/Z/M flags are present for offset/stride 
calculations.



-- 
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