Compare commits

...

6 Commits

Author SHA1 Message Date
Paul Masurel
643639f14b Introduced geopoint. 2025-12-03 17:05:27 +01:00
Paul Masurel
f85a27068d Introduced geopoint. 2025-12-03 17:05:16 +01:00
Paul Masurel
1619e05bc5 plastic surgery 2025-12-03 16:20:18 +01:00
Paul Masurel
5d03c600ba Added bugfix and unit tests
Removed use of robust.
2025-12-03 15:21:37 +01:00
Paul Masurel
32beb06382 plastic surgery 2025-12-03 13:02:10 +01:00
Paul Masurel
d8bc0e7c99 added doc 2025-12-03 12:41:17 +01:00
16 changed files with 362 additions and 159 deletions

View File

@@ -56,7 +56,6 @@ itertools = "0.14.0"
measure_time = "0.9.0"
arc-swap = "1.5.0"
bon = "3.3.1"
robust = "1.2"
i_triangle = "0.38.0"
columnar = { version = "0.6", path = "./columnar", package = "tantivy-columnar" }

View File

@@ -1,6 +1,8 @@
use geo_types::Point;
use tantivy::collector::TopDocs;
use tantivy::query::SpatialQuery;
use tantivy::schema::{Schema, Value, SPATIAL, STORED, TEXT};
use tantivy::spatial::point::GeoPoint;
use tantivy::{Index, IndexWriter, TantivyDocument};
fn main() -> tantivy::Result<()> {
let mut schema_builder = Schema::builder();
@@ -38,7 +40,16 @@ fn main() -> tantivy::Result<()> {
let field = schema.get_field("geometry").unwrap();
let query = SpatialQuery::new(
field,
[(-99.49, 45.56), (-99.45, 45.59)],
[
GeoPoint {
lon: -99.49,
lat: 45.56,
},
GeoPoint {
lon: -99.45,
lat: 45.59,
},
],
tantivy::query::SpatialQueryType::Intersects,
);
let hits = searcher.search(&query, &TopDocs::with_limit(10).order_by_score())?;

View File

@@ -683,7 +683,7 @@ mod tests {
}
#[test]
fn test_datefastfield() -> crate::Result<()> {
fn test_datefastfield() {
let mut schema_builder = Schema::builder();
let date_field = schema_builder.add_date_field(
"date",
@@ -697,22 +697,28 @@ mod tests {
);
let schema = schema_builder.build();
let index = Index::create_in_ram(schema);
let mut index_writer = index.writer_for_tests()?;
let mut index_writer = index.writer_for_tests().unwrap();
index_writer.set_merge_policy(Box::new(NoMergePolicy));
index_writer.add_document(doc!(
date_field => DateTime::from_u64(1i64.to_u64()),
multi_date_field => DateTime::from_u64(2i64.to_u64()),
multi_date_field => DateTime::from_u64(3i64.to_u64())
))?;
index_writer.add_document(doc!(
date_field => DateTime::from_u64(4i64.to_u64())
))?;
index_writer.add_document(doc!(
multi_date_field => DateTime::from_u64(5i64.to_u64()),
multi_date_field => DateTime::from_u64(6i64.to_u64())
))?;
index_writer.commit()?;
let reader = index.reader()?;
index_writer
.add_document(doc!(
date_field => DateTime::from_u64(1i64.to_u64()),
multi_date_field => DateTime::from_u64(2i64.to_u64()),
multi_date_field => DateTime::from_u64(3i64.to_u64())
))
.unwrap();
index_writer
.add_document(doc!(
date_field => DateTime::from_u64(4i64.to_u64())
))
.unwrap();
index_writer
.add_document(doc!(
multi_date_field => DateTime::from_u64(5i64.to_u64()),
multi_date_field => DateTime::from_u64(6i64.to_u64())
))
.unwrap();
index_writer.commit().unwrap();
let reader = index.reader().unwrap();
let searcher = reader.searcher();
assert_eq!(searcher.segment_readers().len(), 1);
let segment_reader = searcher.segment_reader(0);
@@ -746,7 +752,6 @@ mod tests {
assert_eq!(dates[0].into_timestamp_nanos(), 5i64);
assert_eq!(dates[1].into_timestamp_nanos(), 6i64);
}
Ok(())
}
#[test]

View File

@@ -180,8 +180,12 @@ impl SegmentReader {
let fast_fields_readers = FastFieldReaders::open(fast_fields_data, schema.clone())?;
let fieldnorm_data = segment.open_read(SegmentComponent::FieldNorms)?;
let fieldnorm_readers = FieldNormReaders::open(fieldnorm_data)?;
let spatial_data = segment.open_read(SegmentComponent::Spatial)?;
let spatial_readers = SpatialReaders::open(spatial_data)?;
let spatial_readers = if schema.contains_spatial_field() {
let spatial_data = segment.open_read(SegmentComponent::Spatial)?;
SpatialReaders::open(spatial_data)?
} else {
SpatialReaders::empty()
};
let original_bitset = if segment.meta().has_deletes() {
let alive_doc_file_slice = segment.open_read(SegmentComponent::Delete)?;

View File

@@ -1,5 +1,5 @@
use std::collections::HashMap;
use std::io::Write;
use std::io::{BufWriter, Write};
use std::sync::Arc;
use columnar::{
@@ -175,6 +175,7 @@ impl IndexMerger {
let mut readers = vec![];
for (segment, new_alive_bitset_opt) in segments.iter().zip(alive_bitset_opt) {
if segment.meta().num_docs() > 0 {
dbg!("segment");
let reader =
SegmentReader::open_with_custom_alive_set(segment, new_alive_bitset_opt)?;
readers.push(reader);
@@ -530,7 +531,24 @@ impl IndexMerger {
serializer: &mut SegmentSerializer,
doc_id_mapping: &SegmentDocIdMapping,
) -> crate::Result<()> {
/// We need to rebuild a BKD-tree based off the list of triangles.
///
/// Because the data can be large, we do this by writing the sequence of triangles to
/// disk, and mmapping it as mutable slice, and calling the same code as what
/// is done for the segment serialization.
///
/// The OS is in charge of deciding how to handle its page cache.
/// This is the same as what would have happened with swapping,
/// except by explicitly mapping the file, the OS is more likely to
/// swap, the memory will not be accounted as anonymous memory,
/// swap space is reserved etc.
use crate::spatial::bkd::Segment;
let Some(mut spatial_serializer) = serializer.extract_spatial_serializer() else {
// The schema does not contain any spatial field.
return Ok(());
};
let mut segment_mappings: Vec<Vec<Option<DocId>>> = Vec::new();
for reader in &self.readers {
let max_doc = reader.max_doc();
@@ -549,10 +567,10 @@ impl IndexMerger {
}
for (segment_ord, reader) in self.readers.iter().enumerate() {
for (field, temp_file) in &mut temp_files {
let mut buf_temp_file = BufWriter::new(temp_file);
let spatial_readers = reader.spatial_fields();
let spatial_reader = match spatial_readers.get_field(*field)? {
Some(reader) => reader,
None => continue,
let Some(spatial_reader) = spatial_readers.get_field(*field)? else {
continue;
};
let segment = Segment::new(spatial_reader.get_bytes());
for triangle_result in LeafPageIterator::new(&segment) {
@@ -561,33 +579,33 @@ impl IndexMerger {
if let Some(new_doc_id) =
segment_mappings[segment_ord][triangle.doc_id as usize]
{
// This is really just a temporary file, not meant to be portable, so we
// use native endianness here.
for &word in &triangle.words {
temp_file.write_all(&word.to_le_bytes())?;
buf_temp_file.write_all(&word.to_ne_bytes())?;
}
temp_file.write_all(&new_doc_id.to_le_bytes())?;
buf_temp_file.write_all(&new_doc_id.to_ne_bytes())?;
}
}
}
buf_temp_file.flush()?;
// No need to fsync here. This file is not here for persistency.
}
}
if let Some(mut spatial_serializer) = serializer.extract_spatial_serializer() {
for (field, mut temp_file) in temp_files {
// Flush and sync triangles.
temp_file.flush()?;
temp_file.as_file_mut().sync_all()?;
// Memory map the triangle file.
use memmap2::MmapOptions;
let mmap = unsafe { MmapOptions::new().map_mut(temp_file.as_file())? };
// Cast to &[Triangle] slice
let triangle_count = mmap.len() / std::mem::size_of::<Triangle>();
let triangles = unsafe {
std::slice::from_raw_parts_mut(mmap.as_ptr() as *mut Triangle, triangle_count)
};
// Get spatial writer and rebuild block kd-tree.
spatial_serializer.serialize_field(field, triangles)?;
}
spatial_serializer.close()?;
for (field, temp_file) in temp_files {
// Memory map the triangle file.
use memmap2::MmapOptions;
let mmap = unsafe { MmapOptions::new().map_mut(temp_file.as_file())? };
// Cast to &[Triangle] slice
let triangle_count = mmap.len() / std::mem::size_of::<Triangle>();
let triangles = unsafe {
std::slice::from_raw_parts_mut(mmap.as_ptr() as *mut Triangle, triangle_count)
};
// Get spatial writer and rebuild block kd-tree.
spatial_serializer.serialize_field(field, triangles)?;
}
spatial_serializer.close()?;
Ok(())
}

View File

@@ -37,15 +37,20 @@ impl SegmentSerializer {
let fieldnorms_write = segment.open_write(SegmentComponent::FieldNorms)?;
let fieldnorms_serializer = FieldNormsSerializer::from_write(fieldnorms_write)?;
let spatial_write = segment.open_write(SegmentComponent::Spatial)?;
let spatial_serializer = SpatialSerializer::from_write(spatial_write)?;
let spatial_serializer: Option<SpatialSerializer> =
if segment.schema().contains_spatial_field() {
let spatial_write = segment.open_write(SegmentComponent::Spatial)?;
Some(SpatialSerializer::from_write(spatial_write)?)
} else {
None
};
let postings_serializer = InvertedIndexSerializer::open(&mut segment)?;
Ok(SegmentSerializer {
segment,
store_writer,
fast_field_write,
spatial_serializer: Some(spatial_serializer),
spatial_serializer,
fieldnorms_serializer: Some(fieldnorms_serializer),
postings_serializer,
})
@@ -70,7 +75,7 @@ impl SegmentSerializer {
&mut self.fast_field_write
}
/// HUSH
/// Accessor to the `SpatialSerializer`
pub fn extract_spatial_serializer(&mut self) -> Option<SpatialSerializer> {
self.spatial_serializer.take()
}

View File

@@ -6,6 +6,7 @@ use crate::query::explanation::does_not_match;
use crate::query::{BitSetDocSet, Explanation, Query, Scorer, Weight};
use crate::schema::Field;
use crate::spatial::bkd::{search_intersects, Segment};
use crate::spatial::point::GeoPoint;
use crate::spatial::writer::as_point_i32;
use crate::{DocId, DocSet, Score, TantivyError, TERMINATED};
@@ -28,7 +29,7 @@ pub struct SpatialQuery {
impl SpatialQuery {
/// HUSH
pub fn new(field: Field, bounds: [(f64, f64); 2], query_type: SpatialQueryType) -> Self {
pub fn new(field: Field, bounds: [GeoPoint; 2], query_type: SpatialQueryType) -> Self {
SpatialQuery {
field,
bounds: [as_point_i32(bounds[0]), as_point_i32(bounds[1])],

View File

@@ -218,9 +218,14 @@ impl SchemaBuilder {
/// Finalize the creation of a `Schema`
/// This will consume your `SchemaBuilder`
pub fn build(self) -> Schema {
let contains_spatial_field = self
.fields
.iter()
.any(|field_entry| field_entry.field_type().value_type() == Type::Spatial);
Schema(Arc::new(InnerSchema {
fields: self.fields,
fields_map: self.fields_map,
contains_spatial_field,
}))
}
}
@@ -228,6 +233,7 @@ impl SchemaBuilder {
struct InnerSchema {
fields: Vec<FieldEntry>,
fields_map: HashMap<String, Field>, // transient
contains_spatial_field: bool,
}
impl PartialEq for InnerSchema {
@@ -378,6 +384,11 @@ impl Schema {
}
Some((field, json_path))
}
/// Returns true if the schema contains a spatial field.
pub(crate) fn contains_spatial_field(&self) -> bool {
self.0.contains_spatial_field
}
}
impl Serialize for Schema {
@@ -405,16 +416,16 @@ impl<'de> Deserialize<'de> for Schema {
fn visit_seq<A>(self, mut seq: A) -> Result<Self::Value, A::Error>
where A: SeqAccess<'de> {
let mut schema = SchemaBuilder {
let mut schema_builder = SchemaBuilder {
fields: Vec::with_capacity(seq.size_hint().unwrap_or(0)),
fields_map: HashMap::with_capacity(seq.size_hint().unwrap_or(0)),
};
while let Some(value) = seq.next_element()? {
schema.add_field(value);
schema_builder.add_field(value);
}
Ok(schema.build())
Ok(schema_builder.build())
}
}
@@ -1030,4 +1041,33 @@ mod tests {
Some((default, "foobar"))
);
}
#[test]
fn test_contains_spatial_field() {
// No spatial field
{
let mut schema_builder = Schema::builder();
schema_builder.add_text_field("title", TEXT);
let schema = schema_builder.build();
assert!(!schema.contains_spatial_field());
// Serialization check
let schema_json = serde_json::to_string(&schema).unwrap();
let schema_deserialized: Schema = serde_json::from_str(&schema_json).unwrap();
assert!(!schema_deserialized.contains_spatial_field());
}
// With spatial field
{
let mut schema_builder = Schema::builder();
schema_builder.add_text_field("title", TEXT);
schema_builder.add_spatial_field("location", SPATIAL);
let schema = schema_builder.build();
assert!(schema.contains_spatial_field());
// Serialization check
let schema_json = serde_json::to_string(&schema).unwrap();
let schema_deserialized: Schema = serde_json::from_str(&schema_json).unwrap();
assert!(schema_deserialized.contains_spatial_field());
}
}
}

View File

@@ -296,6 +296,8 @@ fn write_branch_nodes(
}
}
const VERSION: u16 = 1u16;
/// Builds and serializes a block kd-tree for spatial indexing of triangles.
///
/// Takes a collection of triangles and constructs a complete block kd-tree, writing both the
@@ -314,16 +316,14 @@ pub fn write_block_kd_tree(
triangles: &mut [Triangle],
write: &mut CountingWriter<WritePtr>,
) -> io::Result<()> {
assert_eq!(
triangles.as_ptr() as usize % std::mem::align_of::<Triangle>(),
0
);
write.write_all(&1u16.to_le_bytes())?;
write.write_all(&VERSION.to_le_bytes())?;
let tree = write_leaf_pages(triangles, write)?;
let current = write.written_bytes();
let aligned = (current + 31) & !31;
let aligned = current.next_multiple_of(32);
let padding = aligned - current;
write.write_all(&vec![0u8; padding as usize])?;
write_leaf_nodes(&tree, write)?;
let branch_position = write.written_bytes();
let mut branch_offset: i32 = 0;
@@ -336,15 +336,16 @@ pub fn write_block_kd_tree(
Ok(())
}
fn decompress_leaf(data: &[u8]) -> io::Result<Vec<Triangle>> {
let count = u16::from_le_bytes([data[0], data[1]]) as usize;
let mut offset = 2;
let mut triangles = Vec::with_capacity(count);
offset += decompress::<u32, _>(&data[offset..], count, |_, doc_id| {
fn decompress_leaf(mut data: &[u8]) -> io::Result<Vec<Triangle>> {
use common::BinarySerializable;
let triangle_count: usize = u16::deserialize(&mut data)? as usize;
let mut offset: usize = 0;
let mut triangles: Vec<Triangle> = Vec::with_capacity(triangle_count);
offset += decompress::<u32, _>(&data[offset..], triangle_count, |_, doc_id| {
triangles.push(Triangle::skeleton(doc_id))
})?;
for i in 0..7 {
offset += decompress::<i32, _>(&data[offset..], count, |j, word| {
offset += decompress::<i32, _>(&data[offset..], triangle_count, |j, word| {
triangles[j].words[i] = word
})?;
}

View File

@@ -5,6 +5,7 @@ use std::io::{self, Read, Write};
use common::{BinarySerializable, VInt};
use serde_json::{json, Map, Value};
use crate::spatial::point::GeoPoint;
use crate::spatial::xor::{compress_f64, decompress_f64};
/// HUSH
@@ -26,17 +27,17 @@ pub enum GeometryError {
#[derive(Debug, Clone, PartialEq)]
pub enum Geometry {
/// HUSH
Point((f64, f64)),
Point(GeoPoint),
/// HUSH
MultiPoint(Vec<(f64, f64)>),
MultiPoint(Vec<GeoPoint>),
/// HUSH
LineString(Vec<(f64, f64)>),
LineString(Vec<GeoPoint>),
/// HUSH
MultiLineString(Vec<Vec<(f64, f64)>>),
MultiLineString(Vec<Vec<GeoPoint>>),
/// HUSH
Polygon(Vec<Vec<(f64, f64)>>),
Polygon(Vec<Vec<GeoPoint>>),
/// HUSH
MultiPolygon(Vec<Vec<Vec<(f64, f64)>>>),
MultiPolygon(Vec<Vec<Vec<GeoPoint>>>),
/// HUSH
GeometryCollection(Vec<Self>),
}
@@ -137,23 +138,24 @@ impl Geometry {
}
}
/// HUSH
/// Serialize the geometry to GeoJSON format.
/// https://fr.wikipedia.org/wiki/GeoJSON
pub fn to_geojson(&self) -> Map<String, Value> {
let mut map = Map::new();
match self {
Geometry::Point(point) => {
map.insert("type".to_string(), Value::String("Point".to_string()));
let coords = json!([point.0, point.1]);
let coords = json!([point.lon, point.lat]);
map.insert("coordinates".to_string(), coords);
}
Geometry::MultiPoint(points) => {
map.insert("type".to_string(), Value::String("MultiPoint".to_string()));
let coords: Vec<Value> = points.iter().map(|p| json!([p.0, p.1])).collect();
let coords: Vec<Value> = points.iter().map(|p| json!([p.lon, p.lat])).collect();
map.insert("coordinates".to_string(), Value::Array(coords));
}
Geometry::LineString(line) => {
map.insert("type".to_string(), Value::String("LineString".to_string()));
let coords: Vec<Value> = line.iter().map(|p| json!([p.0, p.1])).collect();
let coords: Vec<Value> = line.iter().map(|p| json!([p.lon, p.lat])).collect();
map.insert("coordinates".to_string(), Value::Array(coords));
}
Geometry::MultiLineString(lines) => {
@@ -163,7 +165,7 @@ impl Geometry {
);
let coords: Vec<Value> = lines
.iter()
.map(|line| Value::Array(line.iter().map(|p| json!([p.0, p.1])).collect()))
.map(|line| Value::Array(line.iter().map(|p| json!([p.lon, p.lat])).collect()))
.collect();
map.insert("coordinates".to_string(), Value::Array(coords));
}
@@ -171,7 +173,7 @@ impl Geometry {
map.insert("type".to_string(), Value::String("Polygon".to_string()));
let coords: Vec<Value> = rings
.iter()
.map(|ring| Value::Array(ring.iter().map(|p| json!([p.0, p.1])).collect()))
.map(|ring| Value::Array(ring.iter().map(|p| json!([p.lon, p.lat])).collect()))
.collect();
map.insert("coordinates".to_string(), Value::Array(coords));
}
@@ -187,7 +189,9 @@ impl Geometry {
polygon
.iter()
.map(|ring| {
Value::Array(ring.iter().map(|p| json!([p.0, p.1])).collect())
Value::Array(
ring.iter().map(|p| json!([p.lon, p.lat])).collect(),
)
})
.collect(),
)
@@ -218,7 +222,7 @@ fn get_coordinates(object: &Map<String, Value>) -> Result<&Value, GeometryError>
Ok(coordinates)
}
fn to_point(value: &Value) -> Result<(f64, f64), GeometryError> {
fn to_point(value: &Value) -> Result<GeoPoint, GeometryError> {
let lonlat = value.as_array().ok_or(GeometryError::InvalidStructure(
"expected 2 element array pair of lon/lat".to_string(),
))?;
@@ -245,10 +249,10 @@ fn to_point(value: &Value) -> Result<(f64, f64), GeometryError> {
lat
)));
}
Ok((lon, lat))
Ok(GeoPoint { lon, lat })
}
fn to_line_string(value: &Value) -> Result<Vec<(f64, f64)>, GeometryError> {
fn to_line_string(value: &Value) -> Result<Vec<GeoPoint>, GeometryError> {
let mut result = Vec::new();
let coordinates = value.as_array().ok_or(GeometryError::InvalidStructure(
"expected an array of lon/lat arrays".to_string(),
@@ -259,7 +263,7 @@ fn to_line_string(value: &Value) -> Result<Vec<(f64, f64)>, GeometryError> {
Ok(result)
}
fn to_multi_line_string(value: &Value) -> Result<Vec<Vec<(f64, f64)>>, GeometryError> {
fn to_multi_line_string(value: &Value) -> Result<Vec<Vec<GeoPoint>>, GeometryError> {
let mut result = Vec::new();
let coordinates = value.as_array().ok_or(GeometryError::InvalidStructure(
"expected an array of an array of lon/lat arrays".to_string(),
@@ -275,8 +279,8 @@ impl BinarySerializable for Geometry {
match self {
Geometry::Point(point) => {
0u8.serialize(writer)?;
point.0.serialize(writer)?;
point.1.serialize(writer)?;
point.lon.serialize(writer)?;
point.lat.serialize(writer)?;
Ok(())
}
Geometry::MultiPoint(points) => {
@@ -289,7 +293,7 @@ impl BinarySerializable for Geometry {
}
Geometry::MultiLineString(multi_line_string) => {
3u8.serialize(writer)?;
serialize_polygon(multi_line_string, writer)
serialize_polygon(&multi_line_string[..], writer)
}
Geometry::Polygon(polygon) => {
4u8.serialize(writer)?;
@@ -309,8 +313,8 @@ impl BinarySerializable for Geometry {
for polygon in multi_polygon {
for ring in polygon {
for point in ring {
lon.push(point.0);
lat.push(point.1);
lon.push(point.lon);
lat.push(point.lat);
}
}
}
@@ -339,7 +343,7 @@ impl BinarySerializable for Geometry {
0 => {
let lon = BinarySerializable::deserialize(reader)?;
let lat = BinarySerializable::deserialize(reader)?;
Ok(Geometry::Point((lon, lat)))
Ok(Geometry::Point(GeoPoint { lon, lat }))
}
1 => Ok(Geometry::MultiPoint(deserialize_line_string(reader)?)),
2 => Ok(Geometry::LineString(deserialize_line_string(reader)?)),
@@ -370,7 +374,10 @@ impl BinarySerializable for Geometry {
for point_count in rings {
let mut ring = Vec::new();
for _ in 0..point_count {
ring.push((lon[offset], lat[offset]));
ring.push(GeoPoint {
lon: lon[offset],
lat: lat[offset],
});
offset += 1;
}
polygon.push(ring);
@@ -395,16 +402,13 @@ impl BinarySerializable for Geometry {
}
}
fn serialize_line_string<W: Write + ?Sized>(
line: &Vec<(f64, f64)>,
writer: &mut W,
) -> io::Result<()> {
fn serialize_line_string<W: Write + ?Sized>(line: &[GeoPoint], writer: &mut W) -> io::Result<()> {
BinarySerializable::serialize(&VInt(line.len() as u64), writer)?;
let mut lon = Vec::new();
let mut lat = Vec::new();
for point in line {
lon.push(point.0);
lat.push(point.1);
lon.push(point.lon);
lat.push(point.lat);
}
let lon = compress_f64(&lon);
let lat = compress_f64(&lat);
@@ -416,23 +420,23 @@ fn serialize_line_string<W: Write + ?Sized>(
}
fn serialize_polygon<W: Write + ?Sized>(
line_string: &Vec<Vec<(f64, f64)>>,
line_string: &[Vec<GeoPoint>],
writer: &mut W,
) -> io::Result<()> {
BinarySerializable::serialize(&VInt(line_string.len() as u64), writer)?;
for ring in line_string {
BinarySerializable::serialize(&VInt(ring.len() as u64), writer)?;
}
let mut lon = Vec::new();
let mut lat = Vec::new();
let mut lon: Vec<f64> = Vec::new();
let mut lat: Vec<f64> = Vec::new();
for ring in line_string {
for point in ring {
lon.push(point.0);
lat.push(point.1);
lon.push(point.lon);
lat.push(point.lat);
}
}
let lon = compress_f64(&lon);
let lat = compress_f64(&lat);
let lon: Vec<u8> = compress_f64(&lon);
let lat: Vec<u8> = compress_f64(&lat);
VInt(lon.len() as u64).serialize(writer)?;
writer.write_all(&lon)?;
VInt(lat.len() as u64).serialize(writer)?;
@@ -440,20 +444,23 @@ fn serialize_polygon<W: Write + ?Sized>(
Ok(())
}
fn deserialize_line_string<R: Read>(reader: &mut R) -> io::Result<Vec<(f64, f64)>> {
fn deserialize_line_string<R: Read>(reader: &mut R) -> io::Result<Vec<GeoPoint>> {
let point_count = VInt::deserialize(reader)?.0 as usize;
let lon_bytes: Vec<u8> = BinarySerializable::deserialize(reader)?;
let lat_bytes: Vec<u8> = BinarySerializable::deserialize(reader)?;
let lon = decompress_f64(&lon_bytes, point_count);
let lat = decompress_f64(&lat_bytes, point_count);
let mut line_string = Vec::new();
let lon: Vec<f64> = decompress_f64(&lon_bytes, point_count);
let lat: Vec<f64> = decompress_f64(&lat_bytes, point_count);
let mut line_string: Vec<GeoPoint> = Vec::new();
for offset in 0..point_count {
line_string.push((lon[offset], lat[offset]));
line_string.push(GeoPoint {
lon: lon[offset],
lat: lat[offset],
});
}
Ok(line_string)
}
fn deserialize_polygon<R: Read>(reader: &mut R) -> io::Result<Vec<Vec<(f64, f64)>>> {
fn deserialize_polygon<R: Read>(reader: &mut R) -> io::Result<Vec<Vec<GeoPoint>>> {
let ring_count = VInt::deserialize(reader)?.0 as usize;
let mut rings = Vec::new();
let mut count = 0;
@@ -464,14 +471,17 @@ fn deserialize_polygon<R: Read>(reader: &mut R) -> io::Result<Vec<Vec<(f64, f64)
}
let lon_bytes: Vec<u8> = BinarySerializable::deserialize(reader)?;
let lat_bytes: Vec<u8> = BinarySerializable::deserialize(reader)?;
let lon = decompress_f64(&lon_bytes, count);
let lat = decompress_f64(&lat_bytes, count);
let mut polygon = Vec::new();
let lon: Vec<f64> = decompress_f64(&lon_bytes, count);
let lat: Vec<f64> = decompress_f64(&lat_bytes, count);
let mut polygon: Vec<Vec<GeoPoint>> = Vec::new();
let mut offset = 0;
for point_count in rings {
let mut ring = Vec::new();
for _ in 0..point_count {
ring.push((lon[offset], lat[offset]));
ring.push(GeoPoint {
lon: lon[offset],
lat: lat[offset],
});
offset += 1;
}
polygon.push(ring);

View File

@@ -3,6 +3,7 @@
pub mod bkd;
pub mod delta;
pub mod geometry;
pub mod point;
pub mod radix_select;
pub mod reader;
pub mod serializer;

8
src/spatial/point.rs Normal file
View File

@@ -0,0 +1,8 @@
/// A point in the geographical coordinate system.
#[derive(Debug, Clone, Copy, PartialEq)]
pub struct GeoPoint {
/// Longitude
pub lon: f64,
/// Latitude
pub lat: f64,
}

View File

@@ -10,12 +10,17 @@ use crate::schema::Field;
use crate::space_usage::PerFieldSpaceUsage;
#[derive(Clone)]
/// HUSH
pub struct SpatialReaders {
data: Arc<CompositeFile>,
}
impl SpatialReaders {
pub fn empty() -> SpatialReaders {
SpatialReaders {
data: Arc::new(CompositeFile::empty()),
}
}
/// Creates a field norm reader.
pub fn open(file: FileSlice) -> crate::Result<SpatialReaders> {
let data = CompositeFile::open(&file)?;

View File

@@ -6,7 +6,9 @@
//! recovery when needed.
use i_triangle::advanced::delaunay::IntDelaunay;
use robust::{orient2d, Coord};
use i_triangle::i_overlay::i_float::int::point::IntPoint;
use crate::DocId;
const MINY_MINX_MAXY_MAXX_Y_X: i32 = 0;
const MINY_MINX_Y_X_MAXY_MAXX: i32 = 1;
@@ -58,7 +60,7 @@ pub struct Triangle {
/// and a reconstruction code.
pub words: [i32; 7],
/// The id of the document associated with this triangle.
pub doc_id: u32,
pub doc_id: DocId,
}
impl Triangle {
@@ -69,6 +71,9 @@ impl Triangle {
/// indicating which edges are polygon boundaries. Returns a triangle struct with the bounding
/// box in the first four words as `[min_y, min_x, max_y, max_x]`. When decoded, the vertex
/// order may differ from the original input to `new()` due to normalized rotation.
///
/// The edge boundary flags are here to express whether an edge is part of the boundaries
/// in the tesselation of the larger polygon it belongs to.
pub fn new(doc_id: u32, triangle: [i32; 6], boundaries: [bool; 3]) -> Self {
let mut ay = triangle[0];
let mut ax = triangle[1];
@@ -135,18 +140,18 @@ impl Triangle {
}
}
}
// change orientation if CW
if orient2d(
Coord { y: ay, x: ax },
Coord { y: by, x: bx },
Coord { y: cy, x: cx },
) < 0.0
{
// change orientation if clockwise (CW)
if !is_counter_clockwise(
IntPoint { y: ay, x: ax },
IntPoint { y: by, x: bx },
IntPoint { y: cy, x: cx },
) {
// To change the orientation, we simply swap B and C.
let temp_x = bx;
let temp_y = by;
let temp_boundary = ab;
// ax and ay do not change, ab becomes bc
ab = bc;
ab = ca;
bx = cx;
by = cy;
// bc does not change, ca becomes ab
@@ -189,6 +194,22 @@ impl Triangle {
}
}
/// Builds a degenerated triangle degenerating for a single point.
/// All vertices are that point, and all vertices are boundaries.
pub fn from_point(doc_id: DocId, point_x: i32, point_y: i32) -> Triangle {
Triangle::new(
doc_id,
[point_y, point_x, point_y, point_x, point_y, point_x],
[true, true, true],
)
}
/// Builds a degenerated triangle for a segment.
/// Line segment AB is represented as the triangle ABA.
pub fn from_line_segment(doc_id: DocId, a_x: i32, a_y: i32, b_x: i32, b_y: i32) -> Triangle {
Triangle::new(doc_id, [a_y, a_x, b_y, b_x, a_y, a_x], [true, true, true])
}
/// Create a triangle with only the doc_id and the words initialized to zero.
///
/// The doc_id and words in the field are delta-compressed as a series with the doc_id
@@ -328,6 +349,23 @@ pub fn delaunay_to_triangles(doc_id: u32, delaunay: &IntDelaunay, triangles: &mu
}
}
/// Returns true if the path A -> B -> C is Counter-Clockwise (CCW) or collinear.
/// Returns false if it is Clockwise (CW).
#[inline(always)]
fn is_counter_clockwise(a: IntPoint, b: IntPoint, c: IntPoint) -> bool {
// We calculate the 2D cross product (determinant) of vectors AB and AC.
// Formula: (bx - ax)(cy - ay) - (by - ay)(cx - ax)
// We cast to i64 to prevent overflow, as multiplying two i32s can exceed i32::MAX.
let val = (b.x as i64 - a.x as i64) * (c.y as i64 - a.y as i64)
- (b.y as i64 - a.y as i64) * (c.x as i64 - a.x as i64);
// If the result is positive, the triangle is CCW.
// If negative, it is CW.
// If zero, the points are collinear (we return true in that case).
val >= 0
}
#[cfg(test)]
mod tests {
use i_triangle::i_overlay::i_float::int::point::IntPoint;
@@ -355,6 +393,62 @@ mod tests {
}
}
#[test]
fn test_cw_triangle_boundary_and_coord_flip() {
// 1. Define distinct coordinates for a Clockwise triangle
// Visual layout:
// A(50,40): Top Center-ish
// B(10,60): Bottom Right
// C(20,10): Bottom Left (Has the Minimum X=10)
// Path A->B->C is Clockwise.
let input_coords = [
50, 40, // A (y, x)
10, 60, // B
20, 10, // C
];
// 2. Define Boundaries [ab, bc, ca]
// We set BC=true and CA=false.
// The bug (ab=bc) would erroneously put 'true' into the first slot.
// The fix (ab=ca) should put 'false' into the first slot.
let input_bounds = [false, true, false];
// 3. Encode
let triangle = Triangle::new(1, input_coords, input_bounds);
let (decoded_coords, decoded_bounds) = triangle.decode();
// 4. Expected Coordinates
// The internal logic detects CW, swaps B/C to make it CCW:
// A(50,40) -> C(20,10) -> B(10,60)
// Then it rotates to put Min-X first.
// Min X is 10 (Vertex C).
// Final Sequence: C -> B -> A
let expected_coords = [
20, 10, // C
10, 60, // B
50, 40, // A
];
// 5. Expected Boundaries
// After Flip (A->C->B):
// Edge AC (was CA) = false
// Edge CB (was BC) = true
// Edge BA (was AB) = false
// Unrotated: [false, true, false]
// After Rotation (shifting to start at C):
// Shift left by 1: [true, false, false]
let expected_bounds = [true, false, false];
assert_eq!(
decoded_coords, expected_coords,
"Coordinates did not decode as expected"
);
assert_eq!(
decoded_bounds, expected_bounds,
"Boundary flags were incorrect (likely swap bug)"
);
}
#[test]
fn degenerate_triangle() {
let test_cases = [

View File

@@ -7,6 +7,7 @@ use i_triangle::int::triangulatable::IntTriangulatable;
use crate::schema::Field;
use crate::spatial::geometry::Geometry;
use crate::spatial::point::GeoPoint;
use crate::spatial::serializer::SpatialSerializer;
use crate::spatial::triangle::{delaunay_to_triangles, Triangle};
use crate::DocId;
@@ -23,27 +24,27 @@ impl SpatialWriter {
let triangles = &mut self.triangles_by_field.entry(field).or_default();
match geometry {
Geometry::Point(point) => {
into_point(triangles, doc_id, point);
append_point(triangles, doc_id, point);
}
Geometry::MultiPoint(multi_point) => {
for point in multi_point {
into_point(triangles, doc_id, point);
append_point(triangles, doc_id, point);
}
}
Geometry::LineString(line_string) => {
into_line_string(triangles, doc_id, line_string);
append_line_string(triangles, doc_id, line_string);
}
Geometry::MultiLineString(multi_line_string) => {
for line_string in multi_line_string {
into_line_string(triangles, doc_id, line_string);
append_line_string(triangles, doc_id, line_string);
}
}
Geometry::Polygon(polygon) => {
into_polygon(triangles, doc_id, polygon);
append_polygon(triangles, doc_id, &polygon);
}
Geometry::MultiPolygon(multi_polygon) => {
for polygon in multi_polygon {
into_polygon(triangles, doc_id, polygon);
append_polygon(triangles, doc_id, &polygon);
}
}
Geometry::GeometryCollection(geometries) => {
@@ -62,7 +63,7 @@ impl SpatialWriter {
.sum()
}
/// HUSH
/// Serializing our field.
pub fn serialize(&mut self, mut serializer: SpatialSerializer) -> io::Result<()> {
for (field, triangles) in &mut self.triangles_by_field {
serializer.serialize_field(*field, triangles)?;
@@ -81,51 +82,43 @@ impl Default for SpatialWriter {
}
}
/// HUSH
pub fn as_point_i32(point: (f64, f64)) -> (i32, i32) {
/// Convert a point of `(longitude, latitude)` to a integer point.
pub fn as_point_i32(point: GeoPoint) -> (i32, i32) {
(
(point.0 / (360.0 / (1i64 << 32) as f64)).floor() as i32,
(point.1 / (180.0 / (1i64 << 32) as f64)).floor() as i32,
(point.lon / (360.0 / (1i64 << 32) as f64)).floor() as i32,
(point.lat / (180.0 / (1i64 << 32) as f64)).floor() as i32,
)
}
fn into_point(triangles: &mut Vec<Triangle>, doc_id: DocId, point: (f64, f64)) {
fn append_point(triangles: &mut Vec<Triangle>, doc_id: DocId, point: GeoPoint) {
let point = as_point_i32(point);
triangles.push(Triangle::new(
doc_id,
[point.1, point.0, point.1, point.0, point.1, point.0],
[true, true, true],
));
triangles.push(Triangle::from_point(doc_id, point.0, point.1));
}
fn into_line_string(triangles: &mut Vec<Triangle>, doc_id: DocId, line_string: Vec<(f64, f64)>) {
fn append_line_string(triangles: &mut Vec<Triangle>, doc_id: DocId, line_string: Vec<GeoPoint>) {
let mut previous = as_point_i32(line_string[0]);
for point in line_string.into_iter().skip(1) {
let point = as_point_i32(point);
triangles.push(Triangle::new(
doc_id,
[
previous.1, previous.0, point.1, point.0, previous.1, previous.0,
],
[true, true, true],
triangles.push(Triangle::from_line_segment(
doc_id, previous.0, previous.1, point.0, point.1,
));
previous = point
}
}
fn into_ring(i_polygon: &mut Vec<Vec<IntPoint>>, ring: Vec<(f64, f64)>) {
let mut i_ring = Vec::new();
for point in ring {
fn append_ring(i_polygon: &mut Vec<Vec<IntPoint>>, ring: &[GeoPoint]) {
let mut i_ring = Vec::with_capacity(ring.len() + 1);
for &point in ring {
let point = as_point_i32(point);
i_ring.push(IntPoint::new(point.0, point.1));
}
i_polygon.push(i_ring);
}
fn into_polygon(triangles: &mut Vec<Triangle>, doc_id: DocId, polygon: Vec<Vec<(f64, f64)>>) {
let mut i_polygon = Vec::new();
fn append_polygon(triangles: &mut Vec<Triangle>, doc_id: DocId, polygon: &[Vec<GeoPoint>]) {
let mut i_polygon: Vec<Vec<IntPoint>> = Vec::new();
for ring in polygon {
into_ring(&mut i_polygon, ring);
append_ring(&mut i_polygon, ring);
}
let delaunay = i_polygon.triangulate().into_delaunay();
delaunay_to_triangles(doc_id, &delaunay, triangles);

View File

@@ -18,7 +18,7 @@
//! Unlike delta.rs which uses arithmetic deltas for i32 spatial coordinates in the block kd-tree,
//! this module operates on f64 bit patterns directly to preserve exact floating-point values for
//! returning to users.
use std::io::{Cursor, Read};
use std::io::Read;
use common::VInt;
@@ -34,8 +34,8 @@ pub fn compress_f64(values: &[f64]) -> Vec<u8> {
if values.is_empty() {
return Vec::new();
}
let mut output = Vec::new();
let mut previous = values[0].to_bits();
let mut output: Vec<u8> = Vec::new();
let mut previous: u64 = f64_to_le(values[0]);
output.extend_from_slice(&previous.to_le_bytes());
for &value in &values[1..] {
let bits = value.to_bits();
@@ -46,13 +46,21 @@ pub fn compress_f64(values: &[f64]) -> Vec<u8> {
if output.len() >= values.len() * 8 {
let mut output = Vec::with_capacity(values.len() * 8);
for &value in values {
output.extend_from_slice(&value.to_bits().to_le_bytes());
output.extend_from_slice(&f64_to_le(value).to_le_bytes());
}
return output;
}
output
}
fn f64_to_le(value: f64) -> u64 {
u64::from_le_bytes(value.to_le_bytes())
}
fn f64_from_le(value: u64) -> f64 {
f64::from_le_bytes(value.to_le_bytes())
}
/// Decompresses f64 coordinates from XOR delta or raw encoding.
///
/// Detects compression format by byte length - if `bytes.len() == count * 8`, data is raw and
@@ -60,16 +68,16 @@ pub fn compress_f64(values: &[f64]) -> Vec<u8> {
/// reconstructing the original sequence.
///
/// Returns exact f64 values that were passed to `compress_f64()`.
pub fn decompress_f64(bytes: &[u8], count: usize) -> Vec<f64> {
pub fn decompress_f64(mut bytes: &[u8], count: usize) -> Vec<f64> {
let mut values = Vec::with_capacity(count);
if bytes.len() == count * 8 {
for i in 0..count {
let bits = u64::from_le_bytes(bytes[i * 8..(i + 1) * 8].try_into().unwrap());
values.push(f64::from_bits(bits));
values.push(f64_from_le(bits));
}
return values;
}
let mut cursor = Cursor::new(bytes);
let mut cursor: &mut &[u8] = &mut bytes;
// Read first value (raw 8 bytes)
let mut first_bytes = [0u8; 8];