Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
6a5c98c
Implement Geography type and migration
zhangfengcdt Apr 7, 2026
c511380
Add WKBGeography class that stores WKB bytes as the primary represent…
zhangfengcdt Apr 7, 2026
a58fcb2
New Geography metric functions added:
zhangfengcdt Apr 7, 2026
33134a6
implements more st functions for geography
zhangfengcdt Apr 7, 2026
307f37c
implement ST_Contains, ST_Intersects, ST_Equals
zhangfengcdt Apr 7, 2026
86b65da
update plan document
zhangfengcdt Apr 7, 2026
a3294e5
add jmh benckmarks
zhangfengcdt Apr 7, 2026
56a66ed
add SerializerComparisonBench
zhangfengcdt Apr 7, 2026
4c79336
wip
zhangfengcdt Apr 7, 2026
bf686e7
add all st functions to the benchmarks
zhangfengcdt Apr 7, 2026
a0b4618
uses S2ClosestEdgeQuery for true geometry-to-geometry minimum distance
zhangfengcdt Apr 7, 2026
5252459
update benchmark resutls
zhangfengcdt Apr 7, 2026
1567b68
remove plan document
zhangfengcdt Apr 7, 2026
aa9c093
add random geometry/geography generator with configurable vertex count
zhangfengcdt Apr 7, 2026
73e56b6
create GeographyFunctionIntegrationTest
zhangfengcdt Apr 7, 2026
f5e7459
function tests
zhangfengcdt Apr 7, 2026
8b93733
add spark.sedona.geography.eagerShapeIndex
zhangfengcdt Apr 8, 2026
c133a13
add GeoParquet scenario to the benchmark
zhangfengcdt Apr 8, 2026
d9063a1
clean up comments formating
zhangfengcdt Apr 8, 2026
bbbbb2f
updated Geography-Functions.md
zhangfengcdt Apr 8, 2026
5f346e5
add st_within
zhangfengcdt Apr 8, 2026
349ad64
fix lint errors
zhangfengcdt Apr 8, 2026
ca4783e
fix java style issue
zhangfengcdt Apr 8, 2026
606ac4e
fix pre-commit lint
zhangfengcdt Apr 8, 2026
75a6894
fix failed tests
zhangfengcdt Apr 8, 2026
b35bcac
fix tests - updated expected multipolygon WKT to match S2-normalized …
zhangfengcdt Apr 8, 2026
89239fd
fix more tests
zhangfengcdt Apr 8, 2026
0839a97
clean up benchmark code and keep only the core arch and 3 sample st f…
zhangfengcdt Apr 10, 2026
6dbb188
rephrase st classification in doc
zhangfengcdt Apr 10, 2026
383537d
WIP - add WkbS2Shape
zhangfengcdt Apr 15, 2026
0270ab4
Add optimizations and add WkbS2Shape implementing S2Shape
zhangfengcdt Apr 15, 2026
5d9f2e7
add optimization to
zhangfengcdt Apr 16, 2026
6af3e10
address copilot pr comments
zhangfengcdt Apr 20, 2026
2e143f3
fix gh-2810 and other updates as per pr review
zhangfengcdt Apr 20, 2026
544a563
modify svg files
zhangfengcdt Apr 20, 2026
55eb2fc
rever the st_equal order for git to show cleaner diff
zhangfengcdt Apr 21, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,22 @@

public class Distance {

/**
* Compute distance from a single point to a ShapeIndex using PointTarget. This avoids building a
* ShapeIndex for the point side — only the complex geometry needs an index.
*/
public static double S2_distancePointToIndex(S2Point point, ShapeIndexGeography geo) {
S2ClosestEdgeQuery query = S2ClosestEdgeQuery.builder().build(geo.shapeIndex);
S2ClosestEdgeQuery.PointTarget<S1ChordAngle> target =
new S2ClosestEdgeQuery.PointTarget<>(point);
Optional<S2BestEdgesQueryBase.Result> result = query.findClosestEdge(target);
if (!result.isPresent()) {
return Double.POSITIVE_INFINITY;
}
S1ChordAngle chordAngle = (S1ChordAngle) result.get().distance();
return chordAngle.toAngle().radians();
}

public double S2_distance(ShapeIndexGeography geo1, ShapeIndexGeography geo2) {
S2ShapeIndex index1 = geo1.shapeIndex;
S2ShapeIndex index2 = geo2.shapeIndex;
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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 java.io.IOException;
import java.util.Arrays;
import org.locationtech.jts.io.ByteOrderValues;

/**
* Serializer for Geography objects using WKB as the primary format. Byte layout: [4-byte SRID
* big-endian][WKB payload].
*/
public class GeographyWKBSerializer {

/** Header size: 4 bytes SRID. */
private static final int HEADER_SIZE = 4;

/**
* Serialize a Geography to bytes using WKB format.
*
* @param geog the Geography to serialize
* @return byte array with format: [SRID 4 bytes big-endian][WKB payload]
*/
public static byte[] serialize(Geography geog) throws IOException {
byte[] wkb;
if (geog instanceof WKBGeography) {
wkb = ((WKBGeography) geog).getWKBBytes();
} else {
WKBWriter writer = new WKBWriter(2, ByteOrderValues.BIG_ENDIAN, false);
wkb = writer.write(geog);
}

byte[] result = new byte[HEADER_SIZE + wkb.length];
int srid = geog.getSRID();
result[0] = (byte) (srid >> 24);
result[1] = (byte) (srid >> 16);
result[2] = (byte) (srid >> 8);
result[3] = (byte) srid;
System.arraycopy(wkb, 0, result, HEADER_SIZE, wkb.length);
return result;
}

/**
* Deserialize bytes to a Geography using the WKB format.
*
* @param buffer the byte array to deserialize
* @return the deserialized Geography
*/
public static Geography deserialize(byte[] buffer) throws IOException {
int srid =
((buffer[0] & 0xFF) << 24)
| ((buffer[1] & 0xFF) << 16)
| ((buffer[2] & 0xFF) << 8)
| (buffer[3] & 0xFF);
byte[] wkb = Arrays.copyOfRange(buffer, HEADER_SIZE, buffer.length);
return WKBGeography.fromWKB(wkb, srid);
}
}
Loading
Loading