From b56e3086639e253f46411bb1e19f8e212592e3dc Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Mon, 27 Jul 2020 15:04:05 -0700 Subject: [PATCH] geo: order geospatial objects by Hilbert Curve index This follows the [PostGIS way](https://info.crunchydata.com/blog/waiting-for-postgis-3-hilbert-geometry-sorting), but does not follow the same encoding mechanism, so we cannot compare results directly. For Geography we re-use the S2 infrastructure and for Geometry we use the google/hilbert library. Release note (sql change): When ordering by geospatial columns, it will now order by the Hilbert Space Curve index so that points which are geographically similar are clustered together. --- go.mod | 1 + go.sum | 2 + pkg/geo/geo.go | 109 +++++++++++++++++-- pkg/geo/geo_test.go | 100 ++++++++++++++++++ pkg/sql/sem/tree/datum.go | 4 +- pkg/sql/sqlbase/column_type_encoding.go | 8 +- pkg/sql/sqlbase/encoded_datum_test.go | 2 +- pkg/util/encoding/encoding.go | 77 ++++++++++---- pkg/util/encoding/encoding_test.go | 133 +++++++++++++++++++----- vendor | 2 +- 10 files changed, 378 insertions(+), 60 deletions(-) diff --git a/go.mod b/go.mod index 95de5439470a..eb1ef30516a0 100644 --- a/go.mod +++ b/go.mod @@ -77,6 +77,7 @@ require ( github.com/google/flatbuffers v1.11.0 github.com/google/go-cmp v0.4.0 github.com/google/go-github v17.0.0+incompatible + github.com/google/hilbert v0.0.0-20181122061418-320f2e35a565 github.com/google/martian v2.1.0+incompatible // indirect github.com/google/pprof v0.0.0-20190109223431-e84dfd68c163 github.com/googleapis/gax-go v2.0.2+incompatible // indirect diff --git a/go.sum b/go.sum index fdd2d848c442..44152c97afca 100644 --- a/go.sum +++ b/go.sum @@ -329,6 +329,8 @@ github.com/google/go-github v17.0.0+incompatible/go.mod h1:zLgOLi98H3fifZn+44m+u github.com/google/go-querystring v1.0.0 h1:Xkwi/a1rcvNg1PPYe5vI8GbeBY/jrVuDX5ASuANWTrk= github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/hilbert v0.0.0-20181122061418-320f2e35a565 h1:KBAlCAY6eLC44FiEwbzEbHnpVlw15iVM4ZK8QpRIp4U= +github.com/google/hilbert v0.0.0-20181122061418-320f2e35a565/go.mod h1:xn6EodFfRzV6j8NXQRPjngeHWlrpOrsZPKuuLRThU1k= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/pprof v0.0.0-20181127221834-b4f47329b966/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= diff --git a/pkg/geo/geo.go b/pkg/geo/geo.go index ccff9510c8ca..11dd111d2266 100644 --- a/pkg/geo/geo.go +++ b/pkg/geo/geo.go @@ -14,6 +14,7 @@ package geo import ( "bytes" "encoding/binary" + "math" "github.com/cockroachdb/cockroach/pkg/geo/geographiclib" "github.com/cockroachdb/cockroach/pkg/geo/geopb" @@ -23,6 +24,7 @@ import ( "github.com/golang/geo/r1" "github.com/golang/geo/s1" "github.com/golang/geo/s2" + "github.com/google/hilbert" "github.com/twpayne/go-geom" "github.com/twpayne/go-geom/encoding/ewkb" ) @@ -276,6 +278,73 @@ func (g *Geometry) CartesianBoundingBox() *CartesianBoundingBox { return &CartesianBoundingBox{BoundingBox: *g.spatialObject.BoundingBox} } +// SpaceCurveIndex returns an uint64 index to use representing an index into a space-filling curve. +// This will return 0 for empty spatial objects, and math.MaxUint64 for any object outside +// the defined bounds of the given SRID projection. +func (g *Geometry) SpaceCurveIndex() uint64 { + bbox := g.CartesianBoundingBox() + if bbox == nil { + return 0 + } + centerX := (bbox.BoundingBox.LoX + bbox.BoundingBox.HiX) / 2 + centerY := (bbox.BoundingBox.LoY + bbox.BoundingBox.HiY) / 2 + // By default, bound by MaxInt32 (we have not typically seen bounds greater than 1B). + bounds := geoprojbase.Bounds{ + MinX: -math.MaxInt32, + MaxX: math.MaxInt32, + MinY: -math.MaxInt32, + MaxY: math.MaxInt32, + } + if proj, ok := geoprojbase.Projection(g.SRID()); ok { + bounds = proj.Bounds + } + // If we're out of bounds, give up and return a large number. + if centerX > bounds.MaxX || centerY > bounds.MaxY || centerX < bounds.MinX || centerY < bounds.MinY { + return math.MaxUint64 + } + + // We need two compute N such that N is a power of 2 and the x, y coordinates are in [0, N-1]. + // The hilbert curve value will be in the interval [0, 2^N-1]. + // The x, y coordinates will be first normalized below to the interval [0, 1], then multiplied + // by the box size to get the projection. + boxSize := 1 << 40 + // Find the longest dimension, add one here to include the maximum length of the box. + biggestBoxLength := math.Max(bounds.MaxX-bounds.MinX, bounds.MaxY-bounds.MinY) + 1 + // If our max length is smaller than the maximum box size, find the biggest power of two + // after the biggestBoxLength. + // TODO(otan): investigate storing this calculation on the projection itself for perf. + if biggestBoxLength < float64(boxSize) { + boxSize = 1 << int(math.Ceil(math.Log2(biggestBoxLength))) + } + h, err := hilbert.NewHilbert(boxSize) + if err != nil { + panic(err) + } + + xPos := int(((centerX - bounds.MinX) / (bounds.MaxX - bounds.MinX)) * float64(boxSize-1)) + yPos := int(((centerY - bounds.MinY) / (bounds.MaxY - bounds.MinY)) * float64(boxSize-1)) + r, err := h.MapInverse(xPos, yPos) + if err != nil { + panic(err) + } + return uint64(r) +} + +// Compare compares a Geometry against another. +// It compares using SpaceCurveIndex, followed by the byte representation of the Geometry. +// This must produce the same ordering as the index mechanism. +func (g *Geometry) Compare(o *Geometry) int { + lhs := g.SpaceCurveIndex() + rhs := o.SpaceCurveIndex() + if lhs > rhs { + return 1 + } + if lhs < rhs { + return -1 + } + return compareSpatialObjectBytes(g.SpatialObject(), o.SpatialObject()) +} + // // Geography // @@ -489,6 +558,35 @@ func (g *Geography) BoundingCap() s2.Cap { return g.BoundingRect().CapBound() } +// SpaceCurveIndex returns an uint64 index to use representing an index into a space-filling curve. +// This will return 0 for empty spatial objects. +func (g *Geography) SpaceCurveIndex() uint64 { + rect := g.BoundingRect() + if rect.IsEmpty() { + return 0 + } + return uint64(s2.CellIDFromLatLng(rect.Center())) +} + +// Compare compares a Geography against another. +// It compares using SpaceCurveIndex, followed by the byte representation of the Geography. +// This must produce the same ordering as the index mechanism. +func (g *Geography) Compare(o *Geography) int { + lhs := g.SpaceCurveIndex() + rhs := o.SpaceCurveIndex() + if lhs > rhs { + return 1 + } + if lhs < rhs { + return -1 + } + return compareSpatialObjectBytes(g.SpatialObject(), o.SpatialObject()) +} + +// +// Common +// + // IsLinearRingCCW returns whether a given linear ring is counter clock wise. // See 2.07 of http://www.faqs.org/faqs/graphics/algorithms-faq/. // "Find the lowest vertex (or, if there is more than one vertex with the same lowest coordinate, @@ -618,10 +716,6 @@ func S2RegionsFromGeomT(geomRepr geom.T, emptyBehavior EmptyBehavior) ([]s2.Regi return regions, nil } -// -// Common -// - // normalizeLngLat normalizes geographical coordinates into a valid range. func normalizeLngLat(lng float64, lat float64) (float64, float64) { if lat > 90 || lat < -90 { @@ -755,9 +849,10 @@ func shapeTypeFromGeomT(t geom.T) (geopb.ShapeType, error) { } } -// CompareSpatialObject compares the SpatialObject. -// This must match the byte ordering that is be produced by encoding.EncodeGeoAscending. -func CompareSpatialObject(lhs geopb.SpatialObject, rhs geopb.SpatialObject) int { +// compareSpatialObjectBytes compares the SpatialObject if they were serialized. +// This is used for comparison operations, and must be kept consistent with the indexing +// encoding. +func compareSpatialObjectBytes(lhs geopb.SpatialObject, rhs geopb.SpatialObject) int { marshalledLHS, err := protoutil.Marshal(&lhs) if err != nil { panic(err) diff --git a/pkg/geo/geo_test.go b/pkg/geo/geo_test.go index dcd637301862..4dc751814bc2 100644 --- a/pkg/geo/geo_test.go +++ b/pkg/geo/geo_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/geo/geopb" "github.com/cockroachdb/errors" "github.com/golang/geo/s2" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/twpayne/go-geom" ) @@ -561,6 +562,105 @@ func TestGeographyAsS2(t *testing.T) { } } +func TestGeographyHash(t *testing.T) { + testCases := []struct { + orderedWKTs []string + srid geopb.SRID + }{ + { + []string{ + "POINT EMPTY", + "POLYGON EMPTY", + "POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + "POINT(-80 80)", + "LINESTRING(0 0, -90 -80)", + }, + 4326, + }, + { + []string{ + "POINT EMPTY", + "POLYGON EMPTY", + "POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + "POINT(-80 80)", + "LINESTRING(0 0, -90 -80)", + }, + 4004, + }, + } + for i, tc := range testCases { + t.Run(strconv.Itoa(i+1), func(t *testing.T) { + previous := uint64(0) + for _, wkt := range tc.orderedWKTs { + t.Run(wkt, func(t *testing.T) { + g, err := ParseGeography(wkt) + require.NoError(t, err) + g, err = g.CloneWithSRID(tc.srid) + require.NoError(t, err) + + h := g.SpaceCurveIndex() + assert.GreaterOrEqual(t, h, previous) + previous = h + }) + } + }) + } +} + +func TestGeometryHash(t *testing.T) { + testCases := []struct { + orderedWKTs []string + srid geopb.SRID + }{ + { + []string{ + "POINT EMPTY", + "POLYGON EMPTY", + "LINESTRING(0 0, -90 -80)", + "POINT(-80 80)", + "POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + }, + 4326, + }, + { + []string{ + "POINT EMPTY", + "POLYGON EMPTY", + "LINESTRING(0 0, -90 -80)", + "POINT(-80 80)", + "POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + }, + 3857, + }, + { + []string{ + "POINT EMPTY", + "POLYGON EMPTY", + "LINESTRING(0 0, -90 -80)", + "POINT(-80 80)", + "POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + }, + 0, + }, + } + for i, tc := range testCases { + t.Run(strconv.Itoa(i+1), func(t *testing.T) { + previous := uint64(0) + for _, wkt := range tc.orderedWKTs { + t.Run(wkt, func(t *testing.T) { + g, err := ParseGeometry(wkt) + require.NoError(t, err) + g, err = g.CloneWithSRID(tc.srid) + require.NoError(t, err) + h := g.SpaceCurveIndex() + assert.GreaterOrEqual(t, h, previous) + previous = h + }) + } + }) + } +} + func TestGeometryAsGeography(t *testing.T) { for _, tc := range []struct { geom string diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go index 2c857715006b..c49872de6c17 100644 --- a/pkg/sql/sem/tree/datum.go +++ b/pkg/sql/sem/tree/datum.go @@ -2766,7 +2766,7 @@ func (d *DGeography) Compare(ctx *EvalContext, other Datum) int { // NULL is less than any non-NULL value. return 1 } - return geo.CompareSpatialObject(d.Geography.SpatialObject(), other.(*DGeography).SpatialObject()) + return d.Geography.Compare(other.(*DGeography).Geography) } // Prev implements the Datum interface. @@ -2874,7 +2874,7 @@ func (d *DGeometry) Compare(ctx *EvalContext, other Datum) int { // NULL is less than any non-NULL value. return 1 } - return geo.CompareSpatialObject(d.Geometry.SpatialObject(), other.(*DGeometry).SpatialObject()) + return d.Geometry.Compare(other.(*DGeometry).Geometry) } // Prev implements the Datum interface. diff --git a/pkg/sql/sqlbase/column_type_encoding.go b/pkg/sql/sqlbase/column_type_encoding.go index 794a7a89bb17..48c07f617f57 100644 --- a/pkg/sql/sqlbase/column_type_encoding.go +++ b/pkg/sql/sqlbase/column_type_encoding.go @@ -97,15 +97,15 @@ func EncodeTableKey(b []byte, val tree.Datum, dir encoding.Direction) ([]byte, e case *tree.DGeography: so := t.Geography.SpatialObject() if dir == encoding.Ascending { - return encoding.EncodeGeoAscending(b, &so) + return encoding.EncodeGeoAscending(b, t.Geography.SpaceCurveIndex(), &so) } - return encoding.EncodeGeoDescending(b, &so) + return encoding.EncodeGeoDescending(b, t.Geography.SpaceCurveIndex(), &so) case *tree.DGeometry: so := t.Geometry.SpatialObject() if dir == encoding.Ascending { - return encoding.EncodeGeoAscending(b, &so) + return encoding.EncodeGeoAscending(b, t.Geometry.SpaceCurveIndex(), &so) } - return encoding.EncodeGeoDescending(b, &so) + return encoding.EncodeGeoDescending(b, t.Geometry.SpaceCurveIndex(), &so) case *tree.DDate: if dir == encoding.Ascending { return encoding.EncodeVarintAscending(b, t.UnixEpochDaysWithOrig()), nil diff --git a/pkg/sql/sqlbase/encoded_datum_test.go b/pkg/sql/sqlbase/encoded_datum_test.go index 35b883610358..1e45d52ad22c 100644 --- a/pkg/sql/sqlbase/encoded_datum_test.go +++ b/pkg/sql/sqlbase/encoded_datum_test.go @@ -208,7 +208,7 @@ func TestEncDatumCompare(t *testing.T) { for _, typ := range types.OidToType { switch typ.Family() { - case types.AnyFamily, types.UnknownFamily, types.ArrayFamily, types.JsonFamily, types.TupleFamily, types.GeometryFamily, types.GeographyFamily: + case types.AnyFamily, types.UnknownFamily, types.ArrayFamily, types.JsonFamily, types.TupleFamily: continue case types.CollatedStringFamily: typ = types.MakeCollatedString(types.String, *RandCollationLocale(rng)) diff --git a/pkg/util/encoding/encoding.go b/pkg/util/encoding/encoding.go index 56d40440c321..f1e37b647e4e 100644 --- a/pkg/util/encoding/encoding.go +++ b/pkg/util/encoding/encoding.go @@ -584,7 +584,7 @@ func EncodeBytesDescending(b []byte, data []byte) []byte { // are appended to r. The remainder of the input buffer and the // decoded []byte are returned. func DecodeBytesAscending(b []byte, r []byte) ([]byte, []byte, error) { - return decodeBytesInternal(b, r, ascendingBytesEscapes, true) + return decodeBytesInternal(b, r, ascendingBytesEscapes, true /* expectMarker */) } // DecodeBytesDescending decodes a []byte value from the input buffer @@ -597,7 +597,7 @@ func DecodeBytesDescending(b []byte, r []byte) ([]byte, []byte, error) { if r == nil { r = []byte{} } - b, r, err := decodeBytesInternal(b, r, descendingBytesEscapes, true) + b, r, err := decodeBytesInternal(b, r, descendingBytesEscapes, true /* expectMarker */) onesComplement(r) return b, r, err } @@ -978,39 +978,55 @@ func decodeTime(b []byte) (r []byte, sec int64, nsec int64, err error) { // EncodeGeoAscending encodes a geopb.SpatialObject value in ascending order and // returns the new buffer. -// TODO(otan): this should ideally just be encoded by {SRID,Shape,Raw Points}. -// EWKB is expensive to encode. However, we don't store this as a PRIMARY KEY -// (this is needed for GROUP BY only for now), so we ignore it for now. -func EncodeGeoAscending(b []byte, g *geopb.SpatialObject) ([]byte, error) { +// It is sorted by the given curve index, followed by the bytes of the spatial object. +func EncodeGeoAscending(b []byte, curveIndex uint64, g *geopb.SpatialObject) ([]byte, error) { + b = append(b, geoMarker) + b = EncodeUint64Ascending(b, curveIndex) + data, err := protoutil.Marshal(g) if err != nil { return nil, err } - b = encodeBytesAscendingWithTerminatorAndPrefix(b, data, ascendingGeoEscapes.escapedTerm, geoMarker) + b = encodeBytesAscendingWithTerminator(b, data, ascendingGeoEscapes.escapedTerm) return b, nil } // EncodeGeoDescending encodes a geopb.SpatialObject value in descending order and // returns the new buffer. -func EncodeGeoDescending(b []byte, g *geopb.SpatialObject) ([]byte, error) { +// It is sorted by the given curve index, followed by the bytes of the spatial object. +func EncodeGeoDescending(b []byte, curveIndex uint64, g *geopb.SpatialObject) ([]byte, error) { + b = append(b, geoDescMarker) + b = EncodeUint64Descending(b, curveIndex) + + data, err := protoutil.Marshal(g) + if err != nil { + return nil, err + } n := len(b) - var err error - b, err = EncodeGeoAscending(b, g) + b = encodeBytesAscendingWithTerminator(b, data, ascendingGeoEscapes.escapedTerm) if err != nil { return nil, err } - b[n] = geoDescMarker - onesComplement(b[n+1:]) + onesComplement(b[n:]) return b, nil } // DecodeGeoAscending decodes a geopb.SpatialObject value that was encoded // in ascending order back into a geopb.SpatialObject. func DecodeGeoAscending(b []byte) ([]byte, geopb.SpatialObject, error) { + if PeekType(b) != Geo { + return nil, geopb.SpatialObject{}, errors.Errorf("did not find Geo marker") + } + b = b[1:] + var err error + b, _, err = DecodeUint64Ascending(b) + if err != nil { + return nil, geopb.SpatialObject{}, err + } + var pbBytes []byte var ret geopb.SpatialObject - var err error - b, pbBytes, err = decodeBytesInternal(b, pbBytes, ascendingGeoEscapes, true) + b, pbBytes, err = decodeBytesInternal(b, pbBytes, ascendingGeoEscapes, false /* expectMarker */) if err != nil { return b, ret, err } @@ -1021,10 +1037,19 @@ func DecodeGeoAscending(b []byte) ([]byte, geopb.SpatialObject, error) { // DecodeGeoDescending decodes a geopb.SpatialObject value that was encoded // in descending order back into a geopb.SpatialObject. func DecodeGeoDescending(b []byte) ([]byte, geopb.SpatialObject, error) { + if PeekType(b) != GeoDesc { + return nil, geopb.SpatialObject{}, errors.Errorf("did not find Geo marker") + } + b = b[1:] + var err error + b, _, err = DecodeUint64Descending(b) + if err != nil { + return nil, geopb.SpatialObject{}, err + } + var pbBytes []byte var ret geopb.SpatialObject - var err error - b, pbBytes, err = decodeBytesInternal(b, pbBytes, descendingGeoEscapes, true) + b, pbBytes, err = decodeBytesInternal(b, pbBytes, descendingGeoEscapes, false /* expectMarker */) if err != nil { return b, ret, err } @@ -1527,13 +1552,29 @@ func PeekLength(b []byte) (int, error) { case bytesMarker: return getBytesLength(b, ascendingBytesEscapes) case geoMarker: - return getBytesLength(b, ascendingGeoEscapes) + // Expect to reserve at least 8 bytes for int64. + if len(b) < 8 { + return 0, errors.Errorf("slice too short for geospatial (%d)", len(b)) + } + ret, err := getBytesLength(b[8:], ascendingGeoEscapes) + if err != nil { + return 0, err + } + return 8 + ret, nil case jsonInvertedIndex: return getJSONInvertedIndexKeyLength(b) case bytesDescMarker: return getBytesLength(b, descendingBytesEscapes) case geoDescMarker: - return getBytesLength(b, descendingGeoEscapes) + // Expect to reserve at least 8 bytes for int64. + if len(b) < 8 { + return 0, errors.Errorf("slice too short for geospatial (%d)", len(b)) + } + ret, err := getBytesLength(b[8:], descendingGeoEscapes) + if err != nil { + return 0, err + } + return 8 + ret, nil case timeMarker, timeTZMarker: return GetMultiVarintLen(b, 2) case durationBigNegMarker, durationMarker, durationBigPosMarker: diff --git a/pkg/util/encoding/encoding_test.go b/pkg/util/encoding/encoding_test.go index 8038612725a5..7b330edfb309 100644 --- a/pkg/util/encoding/encoding_test.go +++ b/pkg/util/encoding/encoding_test.go @@ -16,6 +16,7 @@ import ( "math" "math/rand" "regexp" + "strconv" "testing" "time" @@ -1147,37 +1148,115 @@ func TestEncodeDecodeTimeTZ(t *testing.T) { } } -func TestEncodeDecodeGeo(t *testing.T) { - testCases := []string{ - "SRID=4326;POINT(1.0 1.0)", - "POINT(2.0 2.0)", +func TestEncodeDecodeGeometry(t *testing.T) { + testCases := []struct { + orderedWKTs []string + }{ + { + orderedWKTs: []string{ + "SRID=4326;POLYGON EMPTY", + "SRID=4326;POINT EMPTY", + "SRID=4326;LINESTRING(0 0, -90 -80)", + "SRID=4326;POINT(-80 80)", + "SRID=4326;POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + }, + }, } - for _, tc := range testCases { - t.Run(tc, func(t *testing.T) { - for _, dir := range []Direction{Ascending, Descending} { + for i, tc := range testCases { + for _, dir := range []Direction{Ascending, Descending} { + t.Run(strconv.Itoa(i+1), func(t *testing.T) { t.Run(fmt.Sprintf("dir:%d", dir), func(t *testing.T) { - parsed, err := geo.ParseGeometry(tc) - require.NoError(t, err) - spatialObject := parsed.SpatialObject() - - var b []byte - var decoded geopb.SpatialObject - if dir == Ascending { - b, err = EncodeGeoAscending(b, &spatialObject) - require.NoError(t, err) - _, decoded, err = DecodeGeoAscending(b) - require.NoError(t, err) - } else { - b, err = EncodeGeoDescending(b, &spatialObject) + var lastEncoded []byte + for _, wkt := range tc.orderedWKTs { + parsed, err := geo.ParseGeometry(wkt) require.NoError(t, err) - _, decoded, err = DecodeGeoDescending(b) + spatialObject := parsed.SpatialObject() + + var b []byte + var decoded geopb.SpatialObject + if dir == Ascending { + b, err = EncodeGeoAscending(b, parsed.SpaceCurveIndex(), &spatialObject) + require.NoError(t, err) + _, decoded, err = DecodeGeoAscending(b) + require.NoError(t, err) + } else { + b, err = EncodeGeoDescending(b, parsed.SpaceCurveIndex(), &spatialObject) + require.NoError(t, err) + _, decoded, err = DecodeGeoDescending(b) + require.NoError(t, err) + } + require.Equal(t, spatialObject, decoded) + testPeekLength(t, b) + + if i > 0 { + if dir == Ascending { + assert.Truef(t, bytes.Compare(b, lastEncoded) > 0, "expected %s > %s", tc.orderedWKTs[i], tc.orderedWKTs[i-1]) + } else { + assert.Truef(t, bytes.Compare(b, lastEncoded) < 0, "expected %s < %s", tc.orderedWKTs[i], tc.orderedWKTs[i-1]) + } + } + + lastEncoded = b + } + }) + }) + } + } +} + +func TestEncodeDecodeGeography(t *testing.T) { + testCases := []struct { + orderedWKTs []string + }{ + { + orderedWKTs: []string{ + "SRID=4326;POLYGON EMPTY", + "SRID=4326;POINT EMPTY", + "SRID=4326;POLYGON((0 0, 1 0, 1 1, 0 1, 0 0))", + "SRID=4326;POINT(-80 80)", + "SRID=4326;LINESTRING(0 0, -90 -80)", + }, + }, + } + for i, tc := range testCases { + for _, dir := range []Direction{Ascending, Descending} { + t.Run(strconv.Itoa(i+1), func(t *testing.T) { + t.Run(fmt.Sprintf("dir:%d", dir), func(t *testing.T) { + var lastEncoded []byte + for _, wkt := range tc.orderedWKTs { + parsed, err := geo.ParseGeography(wkt) require.NoError(t, err) + spatialObject := parsed.SpatialObject() + + var b []byte + var decoded geopb.SpatialObject + if dir == Ascending { + b, err = EncodeGeoAscending(b, parsed.SpaceCurveIndex(), &spatialObject) + require.NoError(t, err) + _, decoded, err = DecodeGeoAscending(b) + require.NoError(t, err) + } else { + b, err = EncodeGeoDescending(b, parsed.SpaceCurveIndex(), &spatialObject) + require.NoError(t, err) + _, decoded, err = DecodeGeoDescending(b) + require.NoError(t, err) + } + require.Equal(t, spatialObject, decoded) + testPeekLength(t, b) + + if i > 0 { + if dir == Ascending { + assert.Truef(t, bytes.Compare(b, lastEncoded) > 0, "expected %s > %s", tc.orderedWKTs[i], tc.orderedWKTs[i-1]) + } else { + assert.Truef(t, bytes.Compare(b, lastEncoded) < 0, "expected %s < %s", tc.orderedWKTs[i], tc.orderedWKTs[i-1]) + } + } + + lastEncoded = b } - require.Equal(t, spatialObject, decoded) - testPeekLength(t, b) }) - } - }) + }) + } } } @@ -1259,9 +1338,9 @@ func TestPeekType(t *testing.T) { require.NoError(t, err) encodedDurationDescending, err := EncodeDurationDescending(nil, duration.Duration{}) require.NoError(t, err) - encodedGeoAscending, err := EncodeGeoAscending(nil, &geopb.SpatialObject{}) + encodedGeoAscending, err := EncodeGeoAscending(nil, 0, &geopb.SpatialObject{}) require.NoError(t, err) - encodedGeoDescending, err := EncodeGeoDescending(nil, &geopb.SpatialObject{}) + encodedGeoDescending, err := EncodeGeoDescending(nil, 0, &geopb.SpatialObject{}) require.NoError(t, err) testCases := []struct { enc []byte diff --git a/vendor b/vendor index df563d91a336..61fb6d90b1fb 160000 --- a/vendor +++ b/vendor @@ -1 +1 @@ -Subproject commit df563d91a3364fc6cd3e112f34ae563bfb20c526 +Subproject commit 61fb6d90b1fbd7d2341b0d2ff0180545684eed63