Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
51183: geogen: introduce package to generate random geo types r=sumeerbhola a=otan

Generate random geo types in a new geogen package. This is immediately
utilised for all the randomized testing, including sqlsmith.

Release note: None

51240: sql: implement the `DROP TYPE` command r=rohany a=rohany

This PR implements the `DROP TYPE` command without support for
`CASCADE`. It lays the foundation of the dependency management between
types and tables that use them. With this, we can drop types when a
database is dropped.

Fixes #48363.

Release note: None

51416: rowexec: refactor index join to use JoinReader r=helenmhe a=helenmhe

Previously there was a separate processor specifically for index joins.
This commit instantiates a joinReader processor instead, so that we
can get rid of the indexJoiner processor and indexJoinerNode.

Resolves: #40749

Release note: None

51537: builtins: implement ST_Intersection and ST_Buffer for geography types r=sumeerbhola a=otan

PostGIS uses _ST_BestSRID to approximate a geography as a geometry to
perform ST_Buffer and ST_Intersection. We're obliged to do the same -
comments are inline.

Resolves #48812
Resolves #48389
Resolves #48390
Resolves #48391
Resolves #48398

Release note (sql change): Implements the ST_Buffer and ST_Intersection
functions for Geography types.

Release note (sql change): Implement ST_Intersection for string types.

51923: opt: move code that generates proto explain tree r=RaduBerinde a=RaduBerinde

Move the code that generates the `roachpb.ExplainTreePlanNode` tree
(shown in the UI) to exec/explain. It parallels the existing code for
generating other explain output.

Release note: None

Co-authored-by: Oliver Tan <[email protected]>
Co-authored-by: Rohan Yadav <[email protected]>
Co-authored-by: Helen He <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
5 people committed Jul 28, 2020
6 parents 4c24889 + d261ae2 + 2dbcc20 + ce0bde7 + 971ad40 + 4f9f8ac commit c8559fe
Show file tree
Hide file tree
Showing 40 changed files with 2,511 additions and 1,048 deletions.
44 changes: 44 additions & 0 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -930,6 +930,39 @@ has no relationship with the commit order of concurrent transactions.</p>
<tr><td><a name="st_azimuth"></a><code>st_azimuth(geometry_a: geometry, geometry_b: geometry) &rarr; <a href="float.html">float</a></code></td><td><span class="funcdesc"><p>Returns the azimuth in radians of the segment defined by the given point geometries, or NULL if the two points are coincident.</p>
<p>The azimuth is angle is referenced from north, and is positive clockwise: North = 0; East = π/2; South = π; West = 3π/2.</p>
</span></td></tr>
<tr><td><a name="st_buffer"></a><code>st_buffer(geography: geography, distance: <a href="float.html">float</a>) &rarr; geography</code></td><td><span class="funcdesc"><p>Returns a Geometry that represents all points whose distance is less than or equal to the given distance
from the given Geometry.</p>
<p>This function utilizes the GEOS module.</p>
<p>This operation is done by transforming the object into a Geometry. This occurs by translating
the Geography objects into Geometry objects before applying an LAEA, UTM or Web Mercator
based projection based on the bounding boxes of the given Geography objects. When the result is
calculated, the result is transformed back into a Geography with SRID 4326.</p>
</span></td></tr>
<tr><td><a name="st_buffer"></a><code>st_buffer(geography: geography, distance: <a href="float.html">float</a>, buffer_style_params: <a href="string.html">string</a>) &rarr; geography</code></td><td><span class="funcdesc"><p>Returns a Geometry that represents all points whose distance is less than or equal to the given distance from the
given Geometry.</p>
<p>This variant takes in a space separate parameter string, which will augment the buffer styles. Valid parameters are:</p>
<ul>
<li>quad_segs=&lt;int&gt;, default 8</li>
<li>endcap=&lt;round|flat|butt|square&gt;, default round</li>
<li>join=&lt;round|mitre|miter|bevel&gt;, default round</li>
<li>side=&lt;both|left|right&gt;, default both</li>
<li>mitre_limit=&lt;float&gt;, default 5.0</li>
</ul>
<p>This function utilizes the GEOS module.</p>
<p>This operation is done by transforming the object into a Geometry. This occurs by translating
the Geography objects into Geometry objects before applying an LAEA, UTM or Web Mercator
based projection based on the bounding boxes of the given Geography objects. When the result is
calculated, the result is transformed back into a Geography with SRID 4326.</p>
</span></td></tr>
<tr><td><a name="st_buffer"></a><code>st_buffer(geography: geography, distance: <a href="float.html">float</a>, quad_segs: <a href="int.html">int</a>) &rarr; geography</code></td><td><span class="funcdesc"><p>Returns a Geometry that represents all points whose distance is less than or equal to the given distance from the
given Geometry.</p>
<p>This variant approximates the circle into quad_seg segments per line (the default is 8).</p>
<p>This function utilizes the GEOS module.</p>
<p>This operation is done by transforming the object into a Geometry. This occurs by translating
the Geography objects into Geometry objects before applying an LAEA, UTM or Web Mercator
based projection based on the bounding boxes of the given Geography objects. When the result is
calculated, the result is transformed back into a Geography with SRID 4326.</p>
</span></td></tr>
<tr><td><a name="st_buffer"></a><code>st_buffer(geometry: geometry, distance: <a href="decimal.html">decimal</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns a Geometry that represents all points whose distance is less than or equal to the given distance
from the given Geometry.</p>
<p>This function utilizes the GEOS module.</p>
Expand Down Expand Up @@ -1171,9 +1204,20 @@ Bottom Left.</p>
</span></td></tr>
<tr><td><a name="st_interiorringn"></a><code>st_interiorringn(geometry: geometry, n: <a href="int.html">int</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns the n-th (1-indexed) interior ring of a Polygon as a LineString. Returns NULL if the shape is not a Polygon, or the ring does not exist.</p>
</span></td></tr>
<tr><td><a name="st_intersection"></a><code>st_intersection(geography_a: geography, geography_b: geography) &rarr; geography</code></td><td><span class="funcdesc"><p>Returns the point intersections of the given geographies.</p>
<p>This operation is done by transforming the object into a Geometry. This occurs by translating
the Geography objects into Geometry objects before applying an LAEA, UTM or Web Mercator
based projection based on the bounding boxes of the given Geography objects. When the result is
calculated, the result is transformed back into a Geography with SRID 4326.</p>
<p>This function utilizes the GEOS module.</p>
</span></td></tr>
<tr><td><a name="st_intersection"></a><code>st_intersection(geometry_a: geometry, geometry_b: geometry) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns the point intersections of the given geometries.</p>
<p>This function utilizes the GEOS module.</p>
</span></td></tr>
<tr><td><a name="st_intersection"></a><code>st_intersection(geometry_a_str: <a href="string.html">string</a>, geometry_b_str: <a href="string.html">string</a>) &rarr; geometry</code></td><td><span class="funcdesc"><p>Returns the point intersections of the given geometries.</p>
<p>This function utilizes the GEOS module.</p>
<p>This variant will cast all geometry_str arguments into Geometry types.</p>
</span></td></tr>
<tr><td><a name="st_intersects"></a><code>st_intersects(geography_a: geography, geography_b: geography) &rarr; <a href="bool.html">bool</a></code></td><td><span class="funcdesc"><p>Returns true if geography_a shares any portion of space with geography_b.</p>
<p>The calculations performed are have a precision of 1cm.</p>
<p>This function utilizes the S2 library for spherical calculations.</p>
Expand Down
212 changes: 212 additions & 0 deletions pkg/geo/geogen/geogen.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,212 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package geogen provides utilities for generating various geospatial types.
package geogen

import (
"math"
"math/rand"
"sort"

"github.com/cockroachdb/cockroach/pkg/geo"
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/geo/geoprojbase"
"github.com/cockroachdb/errors"
"github.com/twpayne/go-geom"
)

var validShapeTypes = []geopb.ShapeType{
geopb.ShapeType_Point,
geopb.ShapeType_LineString,
geopb.ShapeType_Polygon,
geopb.ShapeType_MultiPoint,
geopb.ShapeType_MultiLineString,
geopb.ShapeType_MultiPolygon,
geopb.ShapeType_GeometryCollection,
}

// RandomCoord generates a random coord in the given bounds.
func RandomCoord(rng *rand.Rand, min float64, max float64) float64 {
return rng.Float64()*(max-min) + min
}

// RandomValidLinearRingCoords generates a flat float64 array of coordinates that represents
// a completely closed shape that can represent a simple LinearRing. This shape is always valid.
// A LinearRing must have at least 3 points. A point is added at the end to close the ring.
// Implements the algorithm in https://observablehq.com/@tarte0/generate-random-simple-polygon.
func RandomValidLinearRingCoords(
rng *rand.Rand, numPoints int, minX float64, maxX float64, minY float64, maxY float64,
) []geom.Coord {
if numPoints < 3 {
panic(errors.Newf("need at least 3 points, got %d", numPoints))
}
// Generate N random points, and find the center.
coords := make([]geom.Coord, numPoints+1)
var centerX, centerY float64
for i := 0; i < numPoints; i++ {
coords[i] = geom.Coord{
RandomCoord(rng, minX, maxX),
RandomCoord(rng, minY, maxY),
}
centerX += coords[i].X()
centerY += coords[i].Y()
}

centerX /= float64(numPoints)
centerY /= float64(numPoints)

// Sort by the angle of all the points relative to the center.
// Use ascending order of angle to get a CCW loop.
sort.Slice(coords[:numPoints], func(i, j int) bool {
angleI := math.Atan2(coords[i].Y()-centerY, coords[i].X()-centerX)
angleJ := math.Atan2(coords[j].Y()-centerY, coords[j].X()-centerX)
return angleI < angleJ
})

// Append the first coordinate to the end.
coords[numPoints] = coords[0]
return coords
}

// RandomPoint generates a random Point.
func RandomPoint(
rng *rand.Rand, minX float64, maxX float64, minY float64, maxY float64, srid geopb.SRID,
) *geom.Point {
return geom.NewPointFlat(geom.XY, []float64{
RandomCoord(rng, minX, maxX),
RandomCoord(rng, minY, maxY),
}).SetSRID(int(srid))
}

// RandomLineString generates a random LineString.
func RandomLineString(
rng *rand.Rand, minX float64, maxX float64, minY float64, maxY float64, srid geopb.SRID,
) *geom.LineString {
numCoords := 3 + rand.Intn(10)
randCoords := RandomValidLinearRingCoords(rng, numCoords, minX, maxX, minY, maxY)

// Extract a random substring from the LineString by truncating at the ends.
var minTrunc, maxTrunc int
// Ensure we always have at least two points.
for maxTrunc-minTrunc < 2 {
minTrunc, maxTrunc = rand.Intn(numCoords+1), rand.Intn(numCoords+1)
// Ensure maxTrunc >= minTrunc.
if minTrunc > maxTrunc {
minTrunc, maxTrunc = maxTrunc, minTrunc
}
}
return geom.NewLineString(geom.XY).MustSetCoords(randCoords[minTrunc:maxTrunc]).SetSRID(int(srid))
}

// RandomPolygon generates a random Polygon.
func RandomPolygon(
rng *rand.Rand, minX float64, maxX float64, minY float64, maxY float64, srid geopb.SRID,
) *geom.Polygon {
// TODO(otan): generate random holes inside the Polygon.
// Ideas:
// * We can do something like use 4 arbitrary points in the LinearRing to generate a BoundingBox,
// and re-use "PointInLinearRing" to generate N random points inside the 4 points to form
// a "sub" linear ring inside.
// * Generate a random set of polygons, see which ones they fully cover and use that.
return geom.NewPolygon(geom.XY).MustSetCoords([][]geom.Coord{
RandomValidLinearRingCoords(rng, 3+rng.Intn(10), minX, maxX, minY, maxY),
}).SetSRID(int(srid))
}

// RandomGeomT generates a random geom.T object within the given bounds and SRID.
func RandomGeomT(
rng *rand.Rand, minX float64, maxX float64, minY float64, maxY float64, srid geopb.SRID,
) geom.T {
shapeType := validShapeTypes[rng.Intn(len(validShapeTypes))]
switch shapeType {
case geopb.ShapeType_Point:
return RandomPoint(rng, minX, maxX, minY, maxY, srid)
case geopb.ShapeType_LineString:
return RandomLineString(rng, minX, maxX, minY, maxY, srid)
case geopb.ShapeType_Polygon:
return RandomPolygon(rng, minX, maxX, minY, maxY, srid)
case geopb.ShapeType_MultiPoint:
// TODO(otan): add empty points.
ret := geom.NewMultiPoint(geom.XY).SetSRID(int(srid))
num := 1 + rng.Intn(10)
for i := 0; i < num; i++ {
if err := ret.Push(RandomPoint(rng, minX, maxX, minY, maxY, srid)); err != nil {
panic(err)
}
}
return ret
case geopb.ShapeType_MultiLineString:
// TODO(otan): add empty LineStrings.
ret := geom.NewMultiLineString(geom.XY).SetSRID(int(srid))
num := 1 + rng.Intn(10)
for i := 0; i < num; i++ {
if err := ret.Push(RandomLineString(rng, minX, maxX, minY, maxY, srid)); err != nil {
panic(err)
}
}
return ret
case geopb.ShapeType_MultiPolygon:
// TODO(otan): add empty Polygons.
ret := geom.NewMultiPolygon(geom.XY).SetSRID(int(srid))
num := 1 + rng.Intn(10)
for i := 0; i < num; i++ {
if err := ret.Push(RandomPolygon(rng, minX, maxX, minY, maxY, srid)); err != nil {
panic(err)
}
}
return ret
case geopb.ShapeType_GeometryCollection:
ret := geom.NewGeometryCollection().SetSRID(int(srid))
num := 1 + rng.Intn(10)
for i := 0; i < num; i++ {
var shape geom.T
needShape := true
// Keep searching for a non GeometryCollection.
for needShape {
shape = RandomGeomT(rng, minX, maxX, minY, maxY, srid)
_, needShape = shape.(*geom.GeometryCollection)
}
if err := ret.Push(shape); err != nil {
panic(err)
}
}
return ret
}
panic(errors.Newf("unknown shape type: %v", shapeType))
}

// RandomGeometry generates a random Geometry with the given SRID.
func RandomGeometry(rng *rand.Rand, srid geopb.SRID) *geo.Geometry {
minX, maxX := -math.MaxFloat64, math.MaxFloat64
minY, maxY := -math.MaxFloat64, math.MaxFloat64
proj, ok := geoprojbase.Projections[srid]
if ok {
minX, maxX = proj.Bounds.MinX, proj.Bounds.MaxX
minY, maxY = proj.Bounds.MinY, proj.Bounds.MaxY
}
ret, err := geo.NewGeometryFromGeomT(RandomGeomT(rng, minX, maxX, minY, maxY, srid))
if err != nil {
panic(err)
}
return ret
}

// RandomGeography generates a random Geometry with the given SRID.
func RandomGeography(rng *rand.Rand, srid geopb.SRID) *geo.Geography {
// TODO(otan): generate geographies that traverse latitude/longitude boundaries.
minX, maxX := -180.0, 180.0
minY, maxY := -90.0, 90.0
ret, err := geo.NewGeographyFromGeomT(RandomGeomT(rng, minX, maxX, minY, maxY, srid))
if err != nil {
panic(err)
}
return ret
}
69 changes: 69 additions & 0 deletions pkg/geo/geogen/geogen_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,69 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

// Package geogen provides utilities for generating various geospatial types.
package geogen

import (
"strconv"
"testing"

"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/stretchr/testify/require"
"github.com/twpayne/go-geom"
)

const numRuns = 25

func TestRandomValidLinearRingCoords(t *testing.T) {
rng, _ := randutil.NewPseudoRand()

for run := 0; run < numRuns; run++ {
t.Run(strconv.Itoa(run), func(t *testing.T) {
coords := RandomValidLinearRingCoords(rng, 10, -180, 180, -90, 90)
require.Len(t, coords, 10+1)
for _, coord := range coords {
require.True(t, -180 <= coord.X() && coord.X() <= 180)
require.True(t, -90 <= coord.Y() && coord.Y() <= 90)
}
require.Equal(t, coords[0], coords[len(coords)-1])
})
}
}

func TestRandomGeomT(t *testing.T) {
rng, _ := randutil.NewPseudoRand()
for run := 0; run < numRuns; run++ {
t.Run(strconv.Itoa(run), func(t *testing.T) {
g := RandomGeomT(rng, -180, 180, -90, 90, geopb.SRID(run))
require.Equal(t, run, g.SRID())
if gc, ok := g.(*geom.GeometryCollection); ok {
for gcIdx := 0; gcIdx < gc.NumGeoms(); gcIdx++ {
coords := gc.Geom(gcIdx).FlatCoords()
for i := 0; i < len(coords); i += g.Stride() {
x := coords[i]
y := coords[i+1]
require.True(t, -180 <= x && x <= 180)
require.True(t, -90 <= y && y <= 90)
}
}
} else {
coords := g.FlatCoords()
for i := 0; i < len(coords); i += g.Stride() {
x := coords[i]
y := coords[i+1]
require.True(t, -180 <= x && x <= 180)
require.True(t, -90 <= y && y <= 90)
}
}
})
}
}
Loading

0 comments on commit c8559fe

Please sign in to comment.