Skip to content

Commit

Permalink
Merge #63970
Browse files Browse the repository at this point in the history
63970: geoprojbase: refactor everything to use the Projection abstraction r=sumeerbhola a=otan

Previously, we use the projections map directly, which violates
abstraction bounds as we probably want to keep that map private. We
remedy this by moving everything behind the a Projection function
abstraction which prevents this leakage.

Furthermore, make Projection return an error directly. This is marked,
so we are able to reword error messages in other places.

Makes progress on #63969

Release note: None

Co-authored-by: Oliver Tan <[email protected]>
  • Loading branch information
craig[bot] and otan committed Apr 26, 2021
2 parents 40d6cb1 + 32e5ccb commit 86784e0
Show file tree
Hide file tree
Showing 19 changed files with 157 additions and 81 deletions.
6 changes: 3 additions & 3 deletions pkg/geo/encode.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,9 +90,9 @@ const (

// geomToGeoJSONCRS converts a geom to its CRS GeoJSON form.
func geomToGeoJSONCRS(t geom.T, long bool) (*geojson.CRS, error) {
projection, ok := geoprojbase.Projection(geopb.SRID(t.SRID()))
if !ok {
return nil, errors.Newf("unknown SRID: %d", t.SRID())
projection, err := geoprojbase.Projection(geopb.SRID(t.SRID()))
if err != nil {
return nil, err
}
var prop string
if long {
Expand Down
42 changes: 27 additions & 15 deletions pkg/geo/geo.go
Original file line number Diff line number Diff line change
Expand Up @@ -115,8 +115,8 @@ type Geometry struct {
// MakeGeometry returns a new Geometry. Assumes the input EWKB is validated and in little endian.
func MakeGeometry(spatialObject geopb.SpatialObject) (Geometry, error) {
if spatialObject.SRID != 0 {
if _, ok := geoprojbase.Projection(spatialObject.SRID); !ok {
return Geometry{}, errors.Newf("unknown SRID for Geometry: %d", spatialObject.SRID)
if _, err := geoprojbase.Projection(spatialObject.SRID); err != nil {
return Geometry{}, err
}
}
if spatialObject.Type != geopb.SpatialObjectType_GeometryType {
Expand Down Expand Up @@ -342,10 +342,10 @@ func (g *Geometry) BoundingBoxRef() *geopb.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 {
func (g *Geometry) SpaceCurveIndex() (uint64, error) {
bbox := g.CartesianBoundingBox()
if bbox == nil {
return 0
return 0, nil
}
centerX := (bbox.BoundingBox.LoX + bbox.BoundingBox.HiX) / 2
centerY := (bbox.BoundingBox.LoY + bbox.BoundingBox.HiY) / 2
Expand All @@ -356,12 +356,16 @@ func (g *Geometry) SpaceCurveIndex() uint64 {
MinY: math.MinInt32,
MaxY: math.MaxInt32,
}
if proj, ok := geoprojbase.Projection(g.SRID()); ok {
if g.SRID() != 0 {
proj, err := geoprojbase.Projection(g.SRID())
if err != nil {
return 0, err
}
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
return math.MaxUint64, nil
}

const boxLength = 1 << 32
Expand All @@ -372,15 +376,23 @@ func (g *Geometry) SpaceCurveIndex() uint64 {
// hilbertInverse returns values in the interval [0, boxLength^2-1], so return [0, 2^64-1].
xPos := uint64(((centerX - bounds.MinX) / xBounds) * boxLength)
yPos := uint64(((centerY - bounds.MinY) / yBounds) * boxLength)
return hilbertInverse(boxLength, xPos, yPos)
return hilbertInverse(boxLength, xPos, yPos), nil
}

// 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()
lhs, err := g.SpaceCurveIndex()
if err != nil {
// We should always be able to compare a valid geometry.
panic(err)
}
rhs, err := o.SpaceCurveIndex()
if err != nil {
// We should always be able to compare a valid geometry.
panic(err)
}
if lhs > rhs {
return 1
}
Expand All @@ -401,9 +413,9 @@ type Geography struct {

// MakeGeography returns a new Geography. Assumes the input EWKB is validated and in little endian.
func MakeGeography(spatialObject geopb.SpatialObject) (Geography, error) {
projection, ok := geoprojbase.Projection(spatialObject.SRID)
if !ok {
return Geography{}, errors.Newf("unknown SRID for Geography: %d", spatialObject.SRID)
projection, err := geoprojbase.Projection(spatialObject.SRID)
if err != nil {
return Geography{}, err
}
if !projection.IsLatLng {
return Geography{}, errors.Newf(
Expand Down Expand Up @@ -580,9 +592,9 @@ func (g *Geography) ShapeType2D() geopb.ShapeType {

// Spheroid returns the spheroid represented by the given Geography.
func (g *Geography) Spheroid() (*geographiclib.Spheroid, error) {
proj, ok := geoprojbase.Projection(g.SRID())
if !ok {
return nil, errors.Newf("expected spheroid for SRID %d", g.SRID())
proj, err := geoprojbase.Projection(g.SRID())
if err != nil {
return nil, err
}
return proj.Spheroid, nil
}
Expand Down
7 changes: 5 additions & 2 deletions pkg/geo/geo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -646,7 +646,9 @@ func TestGeometrySpaceCurveIndex(t *testing.T) {
t.Run(tc.wkt, func(t *testing.T) {
g, err := ParseGeometry(tc.wkt)
require.NoError(t, err)
require.Equal(t, tc.expected, g.SpaceCurveIndex())
spaceCurveIndex, err := g.SpaceCurveIndex()
require.NoError(t, err)
require.Equal(t, tc.expected, spaceCurveIndex)
})
}

Expand Down Expand Up @@ -694,7 +696,8 @@ func TestGeometrySpaceCurveIndex(t *testing.T) {
require.NoError(t, err)
g, err = g.CloneWithSRID(tc.srid)
require.NoError(t, err)
h := g.SpaceCurveIndex()
h, err := g.SpaceCurveIndex()
require.NoError(t, err)
assert.GreaterOrEqual(t, h, previous)
previous = h
})
Expand Down
7 changes: 5 additions & 2 deletions pkg/geo/geogen/geogen.go
Original file line number Diff line number Diff line change
Expand Up @@ -327,8 +327,11 @@ func RandomGeometry(rng *rand.Rand, srid geopb.SRID) geo.Geometry {
// the given SRID.
func RandomGeometryWithLayout(rng *rand.Rand, srid geopb.SRID, layout geom.Layout) geo.Geometry {
randomBounds := MakeRandomGeomBounds()
proj, ok := geoprojbase.Projections[srid]
if ok {
if srid != 0 {
proj, err := geoprojbase.Projection(srid)
if err != nil {
panic(err)
}
randomBounds.minX, randomBounds.maxX = proj.Bounds.MinX, proj.Bounds.MaxX
randomBounds.minY, randomBounds.maxY = proj.Bounds.MinY, proj.Bounds.MaxY
}
Expand Down
7 changes: 3 additions & 4 deletions pkg/geo/geogfn/best_projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/geo/geoprojbase"
"github.com/cockroachdb/errors"
"github.com/golang/geo/s1"
"github.com/golang/geo/s2"
)
Expand Down Expand Up @@ -131,9 +130,9 @@ func BestGeomProjection(boundingRect s2.Rect) (geoprojbase.Proj4Text, error) {

// getGeomProjection returns the Proj4Text associated with an SRID.
func getGeomProjection(srid geopb.SRID) (geoprojbase.Proj4Text, error) {
proj, ok := geoprojbase.Projection(srid)
if !ok {
return geoprojbase.Proj4Text{}, errors.Newf("unexpected SRID %d", srid)
proj, err := geoprojbase.Projection(srid)
if err != nil {
return geoprojbase.Proj4Text{}, err
}
return proj.Proj4Text, nil
}
12 changes: 6 additions & 6 deletions pkg/geo/geogfn/best_projection_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,20 +27,20 @@ func TestBestGeomProjection(t *testing.T) {
{
"north pole",
s2.RectFromLatLng(s2.LatLngFromDegrees(75, 75)),
geoprojbase.Projections[3574].Proj4Text,
geoprojbase.MustProjection(3574).Proj4Text,
},
{
"south pole",
s2.RectFromLatLng(s2.LatLngFromDegrees(-75, -75)),
geoprojbase.Projections[3409].Proj4Text},
geoprojbase.MustProjection(3409).Proj4Text},
{
"utm 15 on top hemisphere",
s2.RectFromLatLng(s2.LatLngFromDegrees(15, 93)),
geoprojbase.Projections[32646].Proj4Text},
geoprojbase.MustProjection(32646).Proj4Text},
{
"utm -16 on bottom hemisphere",
s2.RectFromLatLng(s2.LatLngFromDegrees(-15, -111)),
geoprojbase.Projections[32712].Proj4Text,
geoprojbase.MustProjection(32712).Proj4Text,
},
{
"LAEA at equator bands (north half)",
Expand Down Expand Up @@ -75,12 +75,12 @@ func TestBestGeomProjection(t *testing.T) {
{
"UTM which should be 32V, but we return 31V as we do not handle UTM exceptions",
s2.RectFromLatLng(s2.LatLngFromDegrees(59.4136, 5.26)),
geoprojbase.Projections[32631].Proj4Text, // Should be 32632
geoprojbase.MustProjection(32631).Proj4Text, // Should be 32632
},
{
"wide example",
s2.RectFromCenterSize(s2.LatLngFromDegrees(0, 0), s2.LatLngFromDegrees(50, 50)),
geoprojbase.Projections[3857].Proj4Text,
geoprojbase.MustProjection(3857).Proj4Text,
},
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/geo/geoindex/s2_geography_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,9 +159,9 @@ func (i *s2GeographyIndex) DWithin(
distanceMeters float64,
useSphereOrSpheroid geogfn.UseSphereOrSpheroid,
) (UnionKeySpans, error) {
projInfo, ok := geoprojbase.Projection(g.SRID())
if !ok {
return nil, errors.Errorf("projection not found for SRID: %d", g.SRID())
projInfo, err := geoprojbase.Projection(g.SRID())
if err != nil {
return nil, err
}
if projInfo.Spheroid == nil {
return nil, errors.Errorf("projection %d does not have spheroid", g.SRID())
Expand Down
6 changes: 3 additions & 3 deletions pkg/geo/geoindex/s2_geometry_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,9 +83,9 @@ func GeometryIndexConfigForSRID(srid geopb.SRID) (*Config, error) {
if srid == 0 {
return DefaultGeometryIndexConfig(), nil
}
p, exists := geoprojbase.Projection(srid)
if !exists {
return nil, errors.Newf("expected definition for SRID %d", srid)
p, err := geoprojbase.Projection(srid)
if err != nil {
return nil, err
}
b := p.Bounds
minX, maxX, minY, maxY := b.MinX, b.MaxX, b.MinY, b.MaxY
Expand Down
9 changes: 4 additions & 5 deletions pkg/geo/geoindex/s2_geometry_index_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,10 +128,10 @@ func TestNoClippingAtSRIDBounds(t *testing.T) {
// Test that indexes that use the SRID bounds don't clip shapes that touch
// those bounds. This test uses point shapes representing the four corners
// of the bounds.
for srid, projInfo := range geoprojbase.Projections {
t.Run(strconv.Itoa(int(srid)), func(t *testing.T) {
for _, projInfo := range geoprojbase.AllProjections() {
t.Run(strconv.Itoa(int(projInfo.SRID)), func(t *testing.T) {
b := projInfo.Bounds
config, err := GeometryIndexConfigForSRID(srid)
config, err := GeometryIndexConfigForSRID(projInfo.SRID)
require.NoError(t, err)
index := NewS2GeometryIndex(*config.S2Geometry)
// Four corners of the bounds, proceeding clockwise from the lower-left.
Expand All @@ -144,9 +144,8 @@ func TestNoClippingAtSRIDBounds(t *testing.T) {
require.NoError(t, err)
require.Equal(t, 1, len(keys))
require.NotEqual(t, Key(exceedsBoundsCellID), keys[0],
"SRID: %d, Point: %f, %f", srid, xCorners[i], yCorners[i])
"SRID: %d, Point: %f, %f", projInfo.SRID, xCorners[i], yCorners[i])
}
})
}

}
1 change: 1 addition & 0 deletions pkg/geo/geoprojbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
deps = [
"//pkg/geo/geographiclib",
"//pkg/geo/geopb",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand Down
43 changes: 38 additions & 5 deletions pkg/geo/geoprojbase/geoprojbase.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,11 @@ package geoprojbase

import (
"bytes"
"sort"

"github.com/cockroachdb/cockroach/pkg/geo/geographiclib"
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/errors"
)

// Proj4Text is the text representation of a PROJ4 transformation.
Expand Down Expand Up @@ -78,9 +80,40 @@ type ProjInfo struct {
Spheroid *geographiclib.Spheroid
}

// Projection returns the ProjInfo identifier for the given SRID, as well as an bool
// indicating whether the projection exists.
func Projection(srid geopb.SRID) (ProjInfo, bool) {
p, exists := Projections[srid]
return p, exists
// ErrProjectionNotFound indicates a project was not found.
var ErrProjectionNotFound error = errors.New("projection not found")

// Projection returns the ProjInfo for the given SRID, as well as an
// error if the projection does not exist.
func Projection(srid geopb.SRID) (ProjInfo, error) {
p, exists := projections[srid]
if !exists {
return ProjInfo{}, errors.Mark(
errors.Newf("projection for SRID %d does not exist", srid),
ErrProjectionNotFound,
)
}
return p, nil
}

// MustProjection returns the ProjInfo for the given SRID, panicking if the
// projection does not exist.
func MustProjection(srid geopb.SRID) ProjInfo {
ret, err := Projection(srid)
if err != nil {
panic(err)
}
return ret
}

// AllProjections returns a sorted list of all projections.
func AllProjections() []ProjInfo {
ret := make([]ProjInfo, 0, len(projections))
for _, p := range projections {
ret = append(ret, p)
}
sort.Slice(ret, func(i, j int) bool {
return ret[i].SRID < ret[j].SRID
})
return ret
}
4 changes: 2 additions & 2 deletions pkg/geo/geoprojbase/projections.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,10 +73,10 @@ var (
spheroidF73CD907E90D = geographiclib.NewSpheroid(6378249.145, 0.0034075462838492874)
)

// Projections is a mapping of SRID to projections.
// projections is a mapping of SRID to projections.
// Use the `Projection` function to obtain one.
// This file is not spell checked.
var Projections = map[geopb.SRID]ProjInfo{
var projections = map[geopb.SRID]ProjInfo{
2000: {
SRID: 2000,
AuthName: "EPSG",
Expand Down
2 changes: 1 addition & 1 deletion pkg/geo/geoprojbase/projections_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ import (
)

func TestProjections(t *testing.T) {
for srid, proj := range Projections {
for srid, proj := range projections {
t.Run(strconv.Itoa(int(srid)), func(t *testing.T) {
require.NotEqual(t, Bounds{}, proj.Bounds)
require.GreaterOrEqual(t, proj.Bounds.MaxX, proj.Bounds.MinX)
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/geospatial
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,10 @@ statement error type LineString does not match column type Point
INSERT INTO geo_table (id, geom) VALUES
(3, 'SRID=4004;LINESTRING(0.0 0.0, 1.0 2.0)')

statement error unknown SRID for Geometry: 404
statement error projection for SRID 404 does not exist
SELECT 'SRID=404;POINT(1.0 2.0)'::geometry

statement error unknown SRID for Geography: 404
statement error projection for SRID 404 does not exist
SELECT 'SRID=404;POINT(1.0 2.0)'::geography

statement error pq: object type PointZ does not match column type Point
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/opt/invertedidx/geo.go
Original file line number Diff line number Diff line change
Expand Up @@ -681,9 +681,9 @@ func (p *PreFilterer) PreFilter(
geogfn.UseSphereOrSpheroid(tree.MustBeDBool(p.additionalPreFilterParams[1]))
}
// TODO(sumeer): refactor to share code with geogfn.DWithin.
proj, ok := geoprojbase.Projection(fs.srid)
if !ok {
return false, errors.Errorf("cannot compute DWithin on unknown SRID %d", fs.srid)
proj, err := geoprojbase.Projection(fs.srid)
if err != nil {
return false, err
}
angleToExpand := s1.Angle(distance / proj.Spheroid.SphereRadius)
if useSphereOrSpheroid == geogfn.UseSpheroid {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pg_extension.go
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ CREATE TABLE pg_extension.spatial_ref_sys (
proj4text varchar(2048)
)`,
populate: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error {
for _, projection := range geoprojbase.Projections {
for _, projection := range geoprojbase.AllProjections() {
if err := addRow(
tree.NewDInt(tree.DInt(projection.SRID)),
tree.NewDString(projection.AuthName),
Expand Down
8 changes: 6 additions & 2 deletions pkg/sql/rowenc/column_type_encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,10 +110,14 @@ func EncodeTableKey(b []byte, val tree.Datum, dir encoding.Direction) ([]byte, e
return encoding.EncodeGeoDescending(b, t.Geography.SpaceCurveIndex(), so)
case *tree.DGeometry:
so := t.Geometry.SpatialObjectRef()
spaceCurveIndex, err := t.Geometry.SpaceCurveIndex()
if err != nil {
return nil, err
}
if dir == encoding.Ascending {
return encoding.EncodeGeoAscending(b, t.Geometry.SpaceCurveIndex(), so)
return encoding.EncodeGeoAscending(b, spaceCurveIndex, so)
}
return encoding.EncodeGeoDescending(b, t.Geometry.SpaceCurveIndex(), so)
return encoding.EncodeGeoDescending(b, spaceCurveIndex, so)
case *tree.DDate:
if dir == encoding.Ascending {
return encoding.EncodeVarintAscending(b, t.UnixEpochDaysWithOrig()), nil
Expand Down
Loading

0 comments on commit 86784e0

Please sign in to comment.