From f626e512952972db9d798d988eebcf49debd3e16 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Mon, 18 May 2020 17:14:12 -0400 Subject: [PATCH 1/3] rowexec: add batchedInvertedExprEvaluator for inverted index execution This is for evaluating invertedexpr.SpanExpressionProto, both for join queries and single table filtering, using an inverted index. More usage context can be found in #48019 which contains an invertedJoiner that uses an earlier version of this evaluator. Release note: None --- pkg/sql/rowexec/inverted_expr_evaluator.go | 447 ++++++++++++++++++ .../rowexec/inverted_expr_evaluator_test.go | 316 +++++++++++++ 2 files changed, 763 insertions(+) create mode 100644 pkg/sql/rowexec/inverted_expr_evaluator.go create mode 100644 pkg/sql/rowexec/inverted_expr_evaluator_test.go diff --git a/pkg/sql/rowexec/inverted_expr_evaluator.go b/pkg/sql/rowexec/inverted_expr_evaluator.go new file mode 100644 index 000000000000..c911debafa8a --- /dev/null +++ b/pkg/sql/rowexec/inverted_expr_evaluator.go @@ -0,0 +1,447 @@ +// 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 rowexec + +import ( + "bytes" + "sort" + + "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedexpr" +) + +// The abstractions in this file help with evaluating (batches of) +// invertedexpr.SpanExpression. The spans in a SpanExpression represent spans +// of an inverted index, which consists of an inverted column followed by the +// primary key of the table. The set expressions involve union and +// intersection over operands. The operands are sets of primary keys contained +// in the corresponding span. Callers should use batchedInvertedExprEvaluator. +// This evaluator does not do the actual scan -- it is fed the set elements as +// the inverted index is scanned, and routes a set element to all the sets to +// which it belongs (since spans can be overlapping). Once the scan is +// complete, the expressions are evaluated. + +// KeyIndex is used as a set element. It is already de-duped. +type KeyIndex int + +// setContainer is a set of key indexes in increasing order. +type setContainer []KeyIndex + +func (s setContainer) Len() int { + return len(s) +} + +func (s setContainer) Less(i, j int) bool { + return s[i] < s[j] +} + +func (s setContainer) Swap(i, j int) { + s[i], s[j] = s[j], s[i] +} + +func unionSetContainers(a, b setContainer) setContainer { + if len(a) == 0 { + return b + } + if len(b) == 0 { + return a + } + var out setContainer + var i, j int + for i < len(a) && j < len(b) { + if a[i] < b[j] { + out = append(out, a[i]) + i++ + } else if a[i] > b[j] { + out = append(out, b[j]) + j++ + } else { + out = append(out, a[i]) + i++ + j++ + } + } + for ; i < len(a); i++ { + out = append(out, a[i]) + } + for ; j < len(b); j++ { + out = append(out, b[j]) + } + return out +} + +func intersectSetContainers(a, b setContainer) setContainer { + var out setContainer + var i, j int + // TODO(sumeer): when one set is much larger than the other + // it is more efficient to iterate over the smaller set + // and seek into the larger set. + for i < len(a) && j < len(b) { + if a[i] < b[j] { + i++ + } else if a[i] > b[j] { + j++ + } else { + out = append(out, a[i]) + i++ + j++ + } + } + return out +} + +// setExpression follows the structure of SpanExpression. +type setExpression struct { + op invertedexpr.SetOperator + // The index in invertedExprEvaluator.sets + unionSetIndex int + left *setExpression + right *setExpression +} + +type invertedSpan = invertedexpr.SpanExpressionProto_Span +type spanExpression = invertedexpr.SpanExpressionProto_Node + +// The spans in a SpanExpression.FactoredUnionSpans and the corresponding index +// in invertedExprEvaluator.sets. Only populated when FactoredUnionsSpans is +// non-empty. +type spansAndSetIndex struct { + spans []invertedSpan + setIndex int +} + +// invertedExprEvaluator evaluates a single expression. It should not be directly +// used -- see batchedInvertedExprEvaluator. +type invertedExprEvaluator struct { + setExpr *setExpression + // These are initially populated by calls to addIndexRow() as + // the inverted index is scanned. + sets []setContainer + + spansIndex []spansAndSetIndex +} + +func newInvertedExprEvaluator(expr *spanExpression) *invertedExprEvaluator { + eval := &invertedExprEvaluator{} + eval.setExpr = eval.initSetExpr(expr) + return eval +} + +func (ev *invertedExprEvaluator) initSetExpr(expr *spanExpression) *setExpression { + // Assign it an index even if FactoredUnionSpans is empty, since we will + // need it when evaluating. + i := len(ev.sets) + ev.sets = append(ev.sets, nil) + sx := &setExpression{op: expr.Operator, unionSetIndex: i} + if len(expr.FactoredUnionSpans) > 0 { + ev.spansIndex = append(ev.spansIndex, + spansAndSetIndex{spans: expr.FactoredUnionSpans, setIndex: i}) + } + if expr.Left != nil { + sx.left = ev.initSetExpr(expr.Left) + } + if expr.Right != nil { + sx.right = ev.initSetExpr(expr.Right) + } + return sx +} + +// getSpansAndSetIndex returns the spans and corresponding set indexes for +// this expression. The spans are not in sorted order and can be overlapping. +func (ev *invertedExprEvaluator) getSpansAndSetIndex() []spansAndSetIndex { + return ev.spansIndex +} + +// Adds a row to the given set. KeyIndexes are not added in increasing order, +// nor do they represent any ordering of the primary key of the table whose +// inverted index is being read. Also, the same KeyIndex could be added +// repeatedly to a set. +func (ev *invertedExprEvaluator) addIndexRow(setIndex int, keyIndex KeyIndex) { + // If duplicates in a set become a memory problem in this build phase, we + // could do periodic de-duplication as we go. For now, we simply append to + // the slice and de-dup at the start of evaluate(). + ev.sets[setIndex] = append(ev.sets[setIndex], keyIndex) +} + +// Evaluates the expression. The return value is in increasing order +// of KeyIndex. +func (ev *invertedExprEvaluator) evaluate() []KeyIndex { + // Sort and de-dup the sets so that we can efficiently do set operations. + for i, c := range ev.sets { + if len(c) == 0 { + continue + } + sort.Sort(c) + // De-duplicate + set := c[:0] + for j := range c { + if len(set) > 0 && c[j] == set[len(set)-1] { + continue + } + set = append(set, c[j]) + } + ev.sets[i] = set + } + return ev.evaluateSetExpr(ev.setExpr) +} + +func (ev *invertedExprEvaluator) evaluateSetExpr(sx *setExpression) setContainer { + var left, right setContainer + if sx.left != nil { + left = ev.evaluateSetExpr(sx.left) + } + if sx.right != nil { + right = ev.evaluateSetExpr(sx.right) + } + var childrenSet setContainer + switch sx.op { + case invertedexpr.SetUnion: + childrenSet = unionSetContainers(left, right) + case invertedexpr.SetIntersection: + childrenSet = intersectSetContainers(left, right) + } + return unionSetContainers(ev.sets[sx.unionSetIndex], childrenSet) +} + +// Supporting struct for invertedSpanRoutingInfo. +type exprAndSetIndex struct { + // An index into batchedInvertedExprEvaluator.exprEvals. + exprIndex int + // An index into batchedInvertedExprEvaluator.exprEvals[exprIndex].sets. + setIndex int +} + +// invertedSpanRoutingInfo contains the list of exprAndSetIndex pairs that +// need rows from the inverted index span. A []invertedSpanRoutingInfo with +// spans that are sorted and non-overlapping is used to route an added row to +// all the expressions and sets that need that row. +type invertedSpanRoutingInfo struct { + span invertedSpan + exprAndSetIndexList []exprAndSetIndex +} + +// batchedInvertedExprEvaluator is for evaluating one or more expressions. The +// batched evaluator can be reused by calling reset(). In the build phase, +// append expressions directly to exprs. A nil expression is permitted, and is +// just a placeholder that will result in a nil []KeyIndex in evaluate(). +// getSpans() must be called before calls to addIndexRow() even if the result +// of getSpans() is not needed -- it builds the fragmentedSpans used for +// routing the added rows. +type batchedInvertedExprEvaluator struct { + exprs []*invertedexpr.SpanExpressionProto + // The evaluators for all the exprs. + exprEvals []*invertedExprEvaluator + // Spans here are in sorted order and non-overlapping. + fragmentedSpans []invertedSpanRoutingInfo + + // Temporary state used for constructing fragmentedSpans. All spans here + // have the same start key. They are not sorted by end key. + pendingSpans []invertedSpanRoutingInfo +} + +// Helper used in building fragmentedSpans using pendingSpans. pendingSpans +// contains spans with the same start key. This fragments and removes all +// spans up to end key fragmentUntil (or all spans if fragmentUntil == nil). +// +// Example 1: +// pendingSpans contains +// c---g +// c-----i +// c--e +// +// And fragmentUntil = i. Since end keys are exclusive we can fragment and +// remove all spans in pendingSpans. These will be: +// c-e-g +// c-e-g-i +// c-e +// +// For the c-e span, all the exprAndSetIndexList slices for these spans are +// appended since any row in that span needs to be routed to all these +// expressions and sets. For the e-g span only the exprAndSetIndexList slices +// for the top two spans are unioned. +// +// Example 2: +// +// Same pendingSpans, and fragmentUntil = f. The fragments that are generated +// for fragmentedSpans and the remaining spans in pendingSpans are: +// +// fragments remaining +// c-e-f f-g +// c-e-f f-i +// c-e +func (b *batchedInvertedExprEvaluator) fragmentPendingSpans( + fragmentUntil invertedexpr.EncInvertedVal, +) { + // The start keys are the same, so this only sorts in increasing + // order of end keys. + sort.Slice(b.pendingSpans, func(i, j int) bool { + return bytes.Compare(b.pendingSpans[i].span.End, b.pendingSpans[j].span.End) < 0 + }) + for len(b.pendingSpans) > 0 { + if fragmentUntil != nil && bytes.Compare(fragmentUntil, b.pendingSpans[0].span.Start) <= 0 { + break + } + // The prefix of pendingSpans that will be completely consumed when + // the next fragment is constructed. + var removeSize int + // The end of the next fragment. + var end invertedexpr.EncInvertedVal + // The start of the fragment after the next fragment. + var nextStart invertedexpr.EncInvertedVal + if fragmentUntil != nil && bytes.Compare(fragmentUntil, b.pendingSpans[0].span.End) < 0 { + // Can't completely remove any spans from pendingSpans, but a prefix + // of these spans will be removed + removeSize = 0 + end = fragmentUntil + nextStart = end + } else { + // We can remove all spans whose end key is the same as span[0]. + // The end of span[0] is also the end key of this fragment. + removeSize = b.pendingLenWithSameEnd() + end = b.pendingSpans[0].span.End + nextStart = end + } + // The next span to be added to fragmentedSpans. + nextSpan := invertedSpanRoutingInfo{ + span: invertedSpan{ + Start: b.pendingSpans[0].span.Start, + End: end, + }, + } + for i := 0; i < len(b.pendingSpans); i++ { + if i >= removeSize { + // This span is not completely removed so adjust its start. + b.pendingSpans[i].span.Start = nextStart + } + // All spans in pendingSpans contribute to exprAndSetIndexList. + nextSpan.exprAndSetIndexList = + append(nextSpan.exprAndSetIndexList, b.pendingSpans[i].exprAndSetIndexList...) + } + b.fragmentedSpans = append(b.fragmentedSpans, nextSpan) + b.pendingSpans = b.pendingSpans[removeSize:] + if removeSize == 0 { + // fragmentUntil was earlier than the smallest End key in the pending + // spans, so cannot fragment any more. + break + } + } +} + +func (b *batchedInvertedExprEvaluator) pendingLenWithSameEnd() int { + length := 1 + for i := 1; i < len(b.pendingSpans); i++ { + if !bytes.Equal(b.pendingSpans[0].span.End, b.pendingSpans[i].span.End) { + break + } + length++ + } + return length +} + +// getSpans fragments the spans for later routing of rows and returns spans +// representing a union of all the spans (for executing the scan). +func (b *batchedInvertedExprEvaluator) getSpans() []invertedSpan { + if cap(b.exprEvals) < len(b.exprs) { + b.exprEvals = make([]*invertedExprEvaluator, len(b.exprs)) + } else { + b.exprEvals = b.exprEvals[:len(b.exprs)] + } + // Initial spans fetched from all expressions. + var routingSpans []invertedSpanRoutingInfo + for i, expr := range b.exprs { + if expr == nil { + b.exprEvals[i] = nil + continue + } + b.exprEvals[i] = newInvertedExprEvaluator(&expr.Node) + exprSpans := b.exprEvals[i].getSpansAndSetIndex() + for _, spans := range exprSpans { + for _, span := range spans.spans { + routingSpans = append(routingSpans, + invertedSpanRoutingInfo{ + span: span, + exprAndSetIndexList: []exprAndSetIndex{{exprIndex: i, setIndex: spans.setIndex}}, + }, + ) + } + } + } + if len(routingSpans) == 0 { + return nil + } + + // Sort the routingSpans in increasing order of start key, and for equal + // start keys in increasing order of end key. + sort.Slice(routingSpans, func(i, j int) bool { + cmp := bytes.Compare(routingSpans[i].span.Start, routingSpans[j].span.Start) + if cmp == 0 { + cmp = bytes.Compare(routingSpans[i].span.End, routingSpans[j].span.End) + } + return cmp < 0 + }) + + // The union of the spans, which is returned from this function. + var coveringSpans []invertedSpan + currentCoveringSpan := routingSpans[0].span + b.pendingSpans = append(b.pendingSpans, routingSpans[0]) + // This loop does both the union of the routingSpans and fragments the + // routingSpans. + for i := 1; i < len(routingSpans); i++ { + span := routingSpans[i] + if bytes.Compare(b.pendingSpans[0].span.Start, span.span.Start) < 0 { + b.fragmentPendingSpans(span.span.Start) + if bytes.Compare(currentCoveringSpan.End, span.span.Start) < 0 { + coveringSpans = append(coveringSpans, currentCoveringSpan) + currentCoveringSpan = span.span + } else if bytes.Compare(currentCoveringSpan.End, span.span.End) < 0 { + currentCoveringSpan.End = span.span.End + } + } else if bytes.Compare(currentCoveringSpan.End, span.span.End) < 0 { + currentCoveringSpan.End = span.span.End + } + // Add this span to the pending list. + b.pendingSpans = append(b.pendingSpans, span) + } + b.fragmentPendingSpans(nil) + coveringSpans = append(coveringSpans, currentCoveringSpan) + return coveringSpans +} + +// TODO(sumeer): if this will be called in non-decreasing order of enc, +// use that to optimize the binary search. +func (b *batchedInvertedExprEvaluator) addIndexRow( + enc invertedexpr.EncInvertedVal, keyIndex KeyIndex, +) { + i := sort.Search(len(b.fragmentedSpans), func(i int) bool { + return bytes.Compare(b.fragmentedSpans[i].span.Start, enc) > 0 + }) + i-- + for _, elem := range b.fragmentedSpans[i].exprAndSetIndexList { + b.exprEvals[elem.exprIndex].addIndexRow(elem.setIndex, keyIndex) + } +} + +func (b *batchedInvertedExprEvaluator) evaluate() [][]KeyIndex { + result := make([][]KeyIndex, len(b.exprs)) + for i := range b.exprEvals { + if b.exprEvals[i] == nil { + continue + } + result[i] = b.exprEvals[i].evaluate() + } + return result +} + +func (b *batchedInvertedExprEvaluator) reset() { + b.exprs = b.exprs[:0] + b.exprEvals = b.exprEvals[:0] + b.fragmentedSpans = b.fragmentedSpans[:0] + b.pendingSpans = b.pendingSpans[:0] +} diff --git a/pkg/sql/rowexec/inverted_expr_evaluator_test.go b/pkg/sql/rowexec/inverted_expr_evaluator_test.go new file mode 100644 index 000000000000..33728c6e1172 --- /dev/null +++ b/pkg/sql/rowexec/inverted_expr_evaluator_test.go @@ -0,0 +1,316 @@ +// 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 rowexec + +import ( + "fmt" + "math/rand" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/sql/opt/invertedexpr" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +func setToString(set []KeyIndex) string { + var b strings.Builder + for i, elem := range set { + sep := "," + if i == len(set)-1 { + sep = "" + } + fmt.Fprintf(&b, "%d%s", elem, sep) + } + return b.String() +} + +func writeSpan(b *strings.Builder, span invertedSpan) { + fmt.Fprintf(b, "[%s, %s) ", span.Start, span.End) +} + +func spansToString(spans []invertedSpan) string { + var b strings.Builder + for _, elem := range spans { + writeSpan(&b, elem) + } + return b.String() +} + +func spansIndexToString(spans []spansAndSetIndex) string { + var b strings.Builder + for _, elem := range spans { + b.WriteString("spans: ") + for _, s := range elem.spans { + writeSpan(&b, s) + } + fmt.Fprintf(&b, " setIndex: %d\n", elem.setIndex) + } + return b.String() +} + +func fragmentedSpansToString(spans []invertedSpanRoutingInfo) string { + var b strings.Builder + for _, elem := range spans { + b.WriteString("span: ") + writeSpan(&b, elem.span) + b.WriteString(" indexes (expr, set): ") + for _, indexes := range elem.exprAndSetIndexList { + fmt.Fprintf(&b, "(%d, %d) ", indexes.exprIndex, indexes.setIndex) + } + b.WriteString("\n") + } + return b.String() +} + +func keyIndexesToString(indexes [][]KeyIndex) string { + var b strings.Builder + for i, elem := range indexes { + fmt.Fprintf(&b, "%d: ", i) + for _, index := range elem { + fmt.Fprintf(&b, "%d ", index) + } + b.WriteString("\n") + } + return b.String() +} + +func TestSetContainerUnion(t *testing.T) { + defer leaktest.AfterTest(t)() + + type testCase struct { + a setContainer + b setContainer + expected setContainer + } + cases := []testCase{ + {a: nil, b: nil, expected: nil}, + {a: []KeyIndex{5}, b: nil, expected: []KeyIndex{5}}, + {a: []KeyIndex{5}, b: []KeyIndex{2, 12}, expected: []KeyIndex{2, 5, 12}}, + {a: []KeyIndex{2, 5}, b: []KeyIndex{12}, expected: []KeyIndex{2, 5, 12}}, + {a: []KeyIndex{2}, b: []KeyIndex{5, 12}, expected: []KeyIndex{2, 5, 12}}, + {a: []KeyIndex{2, 12}, b: []KeyIndex{2, 5}, expected: []KeyIndex{2, 5, 12}}, + {a: []KeyIndex{2, 5}, b: []KeyIndex{5, 12}, expected: []KeyIndex{2, 5, 12}}, + } + for _, c := range cases { + require.Equal(t, setToString(c.expected), setToString(unionSetContainers(c.a, c.b))) + } +} + +func TestSetContainerIntersection(t *testing.T) { + defer leaktest.AfterTest(t)() + + type testCase struct { + a setContainer + b setContainer + expected setContainer + } + cases := []testCase{ + {a: nil, b: nil, expected: nil}, + {a: []KeyIndex{5}, b: nil, expected: nil}, + {a: []KeyIndex{5}, b: []KeyIndex{2, 12}, expected: nil}, + {a: []KeyIndex{2, 12}, b: []KeyIndex{2, 5}, expected: []KeyIndex{2}}, + {a: []KeyIndex{2, 5}, b: []KeyIndex{5, 12}, expected: []KeyIndex{5}}, + {a: []KeyIndex{2, 5, 17, 25, 30}, b: []KeyIndex{2, 12, 13, 17, 23, 30}, + expected: []KeyIndex{2, 17, 30}}, + } + for _, c := range cases { + require.Equal(t, setToString(c.expected), setToString(intersectSetContainers(c.a, c.b))) + } +} + +type keyAndIndex struct { + key string + index int +} + +// Tests both invertedExprEvaluator and batchedInvertedExprEvaluator. +func TestInvertedExpressionEvaluator(t *testing.T) { + defer leaktest.AfterTest(t)() + + leaf1 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("a"), End: []byte("d")}}, + Operator: invertedexpr.None, + } + leaf2 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("e"), End: []byte("h")}}, + Operator: invertedexpr.None, + } + l1Andl2 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{ + {Start: []byte("i"), End: []byte("j")}, {Start: []byte("k"), End: []byte("n")}}, + Operator: invertedexpr.SetIntersection, + Left: leaf1, + Right: leaf2, + } + leaf3 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("d"), End: []byte("f")}}, + Operator: invertedexpr.None, + } + leaf4 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("a"), End: []byte("c")}}, + Operator: invertedexpr.None, + } + l3Andl4 := &spanExpression{ + FactoredUnionSpans: []invertedSpan{ + {Start: []byte("g"), End: []byte("m")}}, + Operator: invertedexpr.SetIntersection, + Left: leaf3, + Right: leaf4, + } + // In reality, the FactoredUnionSpans of l1Andl2 and l3Andl4 would be moved + // up to expr, by the factoring code in the invertedexpr package. But the + // evaluator does not care, and keeping them separate exercises more code. + exprUnion := &spanExpression{ + Operator: invertedexpr.SetUnion, + Left: l1Andl2, + Right: l3Andl4, + } + + exprIntersection := &spanExpression{ + Operator: invertedexpr.SetIntersection, + Left: l1Andl2, + Right: l3Andl4, + } + + expectedSpansAndSetIndex := "spans: [i, j) [k, n) setIndex: 1\nspans: [a, d) setIndex: 2\n" + + "spans: [e, h) setIndex: 3\nspans: [g, m) setIndex: 4\nspans: [d, f) setIndex: 5\n" + + "spans: [a, c) setIndex: 6\n" + + // Test the getSpansAndSetIndex() method on the invertedExprEvaluator + // directly. The rest of the methods we will only exercise through + // batchedInvertedExprEvaluator. + evalUnion := newInvertedExprEvaluator(exprUnion) + // Indexes are being assigned using a pre-order traversal. + require.Equal(t, expectedSpansAndSetIndex, + spansIndexToString(evalUnion.getSpansAndSetIndex())) + + evalIntersection := newInvertedExprEvaluator(exprIntersection) + require.Equal(t, expectedSpansAndSetIndex, + spansIndexToString(evalIntersection.getSpansAndSetIndex())) + + // The batchedInvertedExprEvaluators will construct their own + // invertedExprEvaluators. + protoUnion := invertedexpr.SpanExpressionProto{Node: *exprUnion} + batchEvalUnion := &batchedInvertedExprEvaluator{ + exprs: []*invertedexpr.SpanExpressionProto{&protoUnion, nil}, + } + protoIntersection := invertedexpr.SpanExpressionProto{Node: *exprIntersection} + batchEvalIntersection := &batchedInvertedExprEvaluator{ + exprs: []*invertedexpr.SpanExpressionProto{&protoIntersection, nil}, + } + expectedSpans := "[a, n) " + expectedFragmentedSpans := + "span: [a, c) indexes (expr, set): (0, 6) (0, 2) \n" + + "span: [c, d) indexes (expr, set): (0, 2) \n" + + "span: [d, e) indexes (expr, set): (0, 5) \n" + + "span: [e, f) indexes (expr, set): (0, 5) (0, 3) \n" + + "span: [f, g) indexes (expr, set): (0, 3) \n" + + "span: [g, h) indexes (expr, set): (0, 3) (0, 4) \n" + + "span: [h, i) indexes (expr, set): (0, 4) \n" + + "span: [i, j) indexes (expr, set): (0, 1) (0, 4) \n" + + "span: [j, k) indexes (expr, set): (0, 4) \n" + + "span: [k, m) indexes (expr, set): (0, 4) (0, 1) \n" + + "span: [m, n) indexes (expr, set): (0, 1) \n" + + require.Equal(t, expectedSpans, spansToString(batchEvalUnion.getSpans())) + require.Equal(t, expectedFragmentedSpans, + fragmentedSpansToString(batchEvalUnion.fragmentedSpans)) + require.Equal(t, expectedSpans, spansToString(batchEvalIntersection.getSpans())) + require.Equal(t, expectedFragmentedSpans, + fragmentedSpansToString(batchEvalIntersection.fragmentedSpans)) + + // Construct the test input. This is hand-constructed to exercise cases where + // intersections sometimes propagate an index across multiple levels. + // This input is not in order of the inverted key since the evaluator does + // not care. In fact, we randomize the input order to exercise the code. + indexRows := []keyAndIndex{{"ii", 0}, {"a", 1}, {"ee", 2}, + {"aa", 3}, {"ab", 4}, {"g", 4}, {"d", 3}, + {"mn", 5}, {"mp", 6}, {"gh", 6}, {"gi", 7}, + {"aaa", 8}, {"e", 8}, {"aaa", 1}} + // These expected values were hand-constructed by evaluating the expression over + // these inputs. + // (([a, d) ∩ [e, h)) U [i, j) U [k, n)) U + // (([d, f) ∩ [a, c)) U [g, m)) + expectedUnion := "0: 0 3 4 5 6 7 8 \n1: \n" + // (([a, d) ∩ [e, h)) U [i, j) U [k, n)) ∩ + // (([d, f) ∩ [a, c)) U [g, m)) + expectedIntersection := "0: 0 4 6 8 \n1: \n" + rand.Shuffle(len(indexRows), func(i, j int) { + indexRows[i], indexRows[j] = indexRows[j], indexRows[i] + }) + for _, elem := range indexRows { + batchEvalUnion.addIndexRow(invertedexpr.EncInvertedVal(elem.key), KeyIndex(elem.index)) + batchEvalIntersection.addIndexRow(invertedexpr.EncInvertedVal(elem.key), KeyIndex(elem.index)) + } + require.Equal(t, expectedUnion, keyIndexesToString(batchEvalUnion.evaluate())) + require.Equal(t, expectedIntersection, keyIndexesToString(batchEvalIntersection.evaluate())) + + // Now do both exprUnion and exprIntersection in a single batch. + batchBoth := batchEvalUnion + batchBoth.reset() + batchBoth.exprs = append(batchBoth.exprs, &protoUnion, &protoIntersection) + batchBoth.getSpans() + for _, elem := range indexRows { + batchBoth.addIndexRow(invertedexpr.EncInvertedVal(elem.key), KeyIndex(elem.index)) + } + require.Equal(t, "0: 0 3 4 5 6 7 8 \n1: 0 4 6 8 \n", + keyIndexesToString(batchBoth.evaluate())) + + // Reset and evaluate nil expressions. + batchBoth.reset() + batchBoth.exprs = append(batchBoth.exprs, nil, nil) + require.Equal(t, 0, len(batchBoth.getSpans())) + require.Equal(t, "0: \n1: \n", keyIndexesToString(batchBoth.evaluate())) +} + +// Test fragmentation for routing when multiple expressions in the batch have +// overlapping spans. +func TestFragmentedSpans(t *testing.T) { + defer leaktest.AfterTest(t)() + + expr1 := invertedexpr.SpanExpressionProto{ + Node: spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("a"), End: []byte("g")}}, + Operator: invertedexpr.None, + }, + } + expr2 := invertedexpr.SpanExpressionProto{ + Node: spanExpression{ + FactoredUnionSpans: []invertedSpan{{Start: []byte("d"), End: []byte("j")}}, + Operator: invertedexpr.None, + }, + } + expr3 := invertedexpr.SpanExpressionProto{ + Node: spanExpression{ + FactoredUnionSpans: []invertedSpan{ + {Start: []byte("e"), End: []byte("f")}, {Start: []byte("i"), End: []byte("l")}, + {Start: []byte("o"), End: []byte("p")}}, + Operator: invertedexpr.None, + }, + } + batchEval := &batchedInvertedExprEvaluator{ + exprs: []*invertedexpr.SpanExpressionProto{&expr1, &expr2, &expr3}, + } + require.Equal(t, "[a, l) [o, p) ", spansToString(batchEval.getSpans())) + require.Equal(t, + "span: [a, d) indexes (expr, set): (0, 0) \n"+ + "span: [d, e) indexes (expr, set): (0, 0) (1, 0) \n"+ + "span: [e, f) indexes (expr, set): (2, 0) (0, 0) (1, 0) \n"+ + "span: [f, g) indexes (expr, set): (0, 0) (1, 0) \n"+ + "span: [g, i) indexes (expr, set): (1, 0) \n"+ + "span: [i, j) indexes (expr, set): (1, 0) (2, 0) \n"+ + "span: [j, l) indexes (expr, set): (2, 0) \n"+ + "span: [o, p) indexes (expr, set): (2, 0) \n", + fragmentedSpansToString(batchEval.fragmentedSpans)) +} + +// TODO(sumeer): randomized inputs for union, intersection and expression evaluation. From 8a4aeedc2f6a8cb5a591155aee7a4c0a137d4d9c Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Tue, 26 May 2020 12:40:37 -0400 Subject: [PATCH 2/3] sql: add the max/min aggregations on ENUMs Fixes #48370. Release note (sql change): Add the max/min aggregation on ENUM types. --- docs/generated/sql/aggregates.md | 4 ++++ pkg/sql/logictest/testdata/logic_test/enums | 16 ++++++++++++++++ pkg/sql/logictest/testdata/logic_test/pg_catalog | 2 ++ pkg/sql/sem/builtins/aggregate_builtins.go | 2 +- 4 files changed, 23 insertions(+), 1 deletion(-) diff --git a/docs/generated/sql/aggregates.md b/docs/generated/sql/aggregates.md index 5c2f9e61d05c..369441759d80 100644 --- a/docs/generated/sql/aggregates.md +++ b/docs/generated/sql/aggregates.md @@ -105,6 +105,8 @@ max(arg1: uuid) → uuid

Identifies the maximum selected value.

+max(arg1: anyenum) → anyelement

Identifies the maximum selected value.

+
max(arg1: collatedstring{*}) → anyelement

Identifies the maximum selected value.

max(arg1: geography) → geography

Identifies the maximum selected value.

@@ -145,6 +147,8 @@
min(arg1: uuid) → uuid

Identifies the minimum selected value.

+min(arg1: anyenum) → anyelement

Identifies the minimum selected value.

+
min(arg1: collatedstring{*}) → anyelement

Identifies the minimum selected value.

min(arg1: geography) → geography

Identifies the minimum selected value.

diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index 8f55d521c4a1..02552c9cc354 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -345,6 +345,22 @@ hello 3 4 1 howdy 5 6 0 hi 10 10 10 +# Test aggregations on ENUM columns. +query TT +SELECT max(x), min(x) FROM enum_agg +---- +hi hello + +# Test that enums without any members can still get an aggregate +# resolved when distributing a flow. +statement ok +CREATE TYPE empty AS ENUM (); +CREATE TABLE empty_enum (x empty) + +query TT +SELECT max(x), min(x) FROM empty_enum +---- +NULL NULL # Regression to ensure that statistics jobs can be run on tables # with user defined types. diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 29c42ad29947..413d28722dd8 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -2092,6 +2092,7 @@ JOIN pg_operator c ON c.oprname = '>' AND b.proargtypes[0] = c.oprleft AND b.pro WHERE (b.proname = 'max' OR b.proname = 'bool_or') AND c.oid = a.aggsortop; ---- oid oprname aggsortop +1224236426 > 1224236426 3636536082 > 3636536082 3636536082 > 3636536082 2948286002 > 2948286002 @@ -2123,6 +2124,7 @@ JOIN pg_operator c ON c.oprname = '<' AND b.proargtypes[0] = c.oprleft AND b.pro WHERE (b.proname = 'min' OR b.proname = 'bool_and' OR b.proname = 'every') AND c.oid = a.aggsortop; ---- oid oprname aggsortop +3859576864 < 3859576864 2134593616 < 2134593616 2134593616 < 2134593616 2134593616 < 2134593616 diff --git a/pkg/sql/sem/builtins/aggregate_builtins.go b/pkg/sql/sem/builtins/aggregate_builtins.go index c823cdcf65b1..8e4bbc33cbd5 100644 --- a/pkg/sql/sem/builtins/aggregate_builtins.go +++ b/pkg/sql/sem/builtins/aggregate_builtins.go @@ -80,7 +80,7 @@ func aggPropsNullableArgs() tree.FunctionProperties { // allMaxMinAggregateTypes contains extra types that aren't in // types.Scalar that the max/min aggregate functions are defined on. var allMaxMinAggregateTypes = append( - []*types.T{types.AnyCollatedString}, + []*types.T{types.AnyCollatedString, types.AnyEnum}, types.Scalar..., ) From 29b60f239da808f942a2efbfa054331779e8a2ae Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 26 May 2020 19:18:40 -0700 Subject: [PATCH 3/3] geo/geo*fn: handle EMPTY cases in GEOMETRY/GEOGRAPHY There are many cases of EMPTY that we need to handle for each method we support. This includes `POINT EMPTY`, `LINESTRING EMPTY` ... `GEOMETRYCOLLECTION EMPTY` (i.e. one for each shape). Furthermore, there's an even more confusing GEOMETRYCOLLECTION with EMPTY shapes, e.g. `GEOMETRYCOLLECTION ( LINESTRING EMPTY ) ` Both GEOS and PostGIS have various bugs in handling these, which have been commented in code and imitated where appropriate. Also bump twpayne/go-geom to pickup the fixed Empty() checks. Release note: None --- pkg/geo/errors.go | 45 ++- pkg/geo/geo.go | 57 +++- pkg/geo/geo_test.go | 48 +++- pkg/geo/geogfn/covers.go | 9 +- pkg/geo/geogfn/covers_test.go | 24 ++ pkg/geo/geogfn/distance.go | 5 +- pkg/geo/geogfn/distance_test.go | 27 ++ pkg/geo/geogfn/dwithin.go | 11 +- pkg/geo/geogfn/dwithin_test.go | 26 ++ pkg/geo/geogfn/intersects.go | 4 +- pkg/geo/geogfn/intersects_test.go | 18 ++ pkg/geo/geogfn/unary_operators.go | 14 +- pkg/geo/geogfn/unary_operators_test.go | 26 ++ pkg/geo/geoindex/s2_geography_index.go | 10 +- pkg/geo/geomfn/distance.go | 43 ++- pkg/geo/geomfn/distance_test.go | 117 ++++++-- pkg/geo/geomfn/geomfn.go | 32 ++- .../logictest/testdata/logic_test/geospatial | 265 +++++++++++++++++- pkg/sql/sem/builtins/geo_builtins.go | 24 +- 19 files changed, 739 insertions(+), 66 deletions(-) diff --git a/pkg/geo/errors.go b/pkg/geo/errors.go index cac23023005a..4c09c9d86302 100644 --- a/pkg/geo/errors.go +++ b/pkg/geo/errors.go @@ -10,7 +10,11 @@ package geo -import "github.com/cockroachdb/errors" +import ( + "fmt" + + "github.com/cockroachdb/errors" +) // NewMismatchingSRIDsError returns the error message for SRIDs of GeospatialTypes // a and b being a mismatch. @@ -23,3 +27,42 @@ func NewMismatchingSRIDsError(a GeospatialType, b GeospatialType) error { b.SRID(), ) } + +// EmptyGeometryError is an error that is returned when the Geometry or any +// parts of its subgeometries are empty. +type EmptyGeometryError struct { + cause error +} + +var _ error = (*EmptyGeometryError)(nil) +var _ errors.SafeDetailer = (*EmptyGeometryError)(nil) +var _ fmt.Formatter = (*EmptyGeometryError)(nil) +var _ errors.Formatter = (*EmptyGeometryError)(nil) + +// Error implements the error interface. +func (w *EmptyGeometryError) Error() string { return w.cause.Error() } + +// Cause implements the errors.SafeDetailer interface. +func (w *EmptyGeometryError) Cause() error { return w.cause } + +// Unwrap implements the SafeDetailer interface. +func (w *EmptyGeometryError) Unwrap() error { return w.cause } + +// SafeDetails implements the SafeDetailer interface. +func (w *EmptyGeometryError) SafeDetails() []string { return []string{w.cause.Error()} } + +// Format implements the errors.Formatter interface. +func (w *EmptyGeometryError) Format(s fmt.State, verb rune) { errors.FormatError(w, s, verb) } + +// FormatError implements the errors.Formatter interface. +func (w *EmptyGeometryError) FormatError(p errors.Printer) (next error) { return w.cause } + +// IsEmptyGeometryError returns true if the error is of type EmptyGeometryError. +func IsEmptyGeometryError(err error) bool { + return errors.HasType(err, &EmptyGeometryError{}) +} + +// NewEmptyGeometryError returns an error indicating an empty geometry has been found. +func NewEmptyGeometryError() *EmptyGeometryError { + return &EmptyGeometryError{cause: errors.Newf("empty shape found")} +} diff --git a/pkg/geo/geo.go b/pkg/geo/geo.go index 7dc4d799c55b..1b2584f2a642 100644 --- a/pkg/geo/geo.go +++ b/pkg/geo/geo.go @@ -24,6 +24,17 @@ import ( // DefaultEWKBEncodingFormat is the default encoding format for EWKB. var DefaultEWKBEncodingFormat = binary.LittleEndian +// EmptyBehavior is the behavior to adopt when an empty Geometry is encountered. +type EmptyBehavior uint8 + +const ( + // EmptyBehaviorError will error with EmptyGeometryError when an empty geometry + // is encountered. + EmptyBehaviorError EmptyBehavior = 0 + // EmptyBehaviorOmit will omit an entry when an empty geometry is encountered. + EmptyBehaviorOmit EmptyBehavior = 1 +) + // // Geospatial Type // @@ -353,14 +364,14 @@ func (g *Geography) Shape() geopb.Shape { return g.SpatialObject.Shape } -// AsS2 converts a given Geography into its S2 form. -func (g *Geography) AsS2() ([]s2.Region, error) { +// AsS2 converts a given Geography into it's S2 form. +func (g *Geography) AsS2(emptyBehavior EmptyBehavior) ([]s2.Region, error) { geomRepr, err := g.AsGeomT() if err != nil { return nil, err } // TODO(otan): convert by reading from EWKB to S2 directly. - return S2RegionsFromGeom(geomRepr), nil + return S2RegionsFromGeom(geomRepr, emptyBehavior) } // isLinearRingCCW returns whether a given linear ring is counter clock wise. @@ -407,8 +418,20 @@ func isLinearRingCCW(linearRing *geom.LinearRing) bool { // S2RegionsFromGeom converts an geom representation of an object // to s2 regions. -func S2RegionsFromGeom(geomRepr geom.T) []s2.Region { +// As S2 does not really handle empty geometries well, we need to ingest emptyBehavior and +// react appropriately. +func S2RegionsFromGeom(geomRepr geom.T, emptyBehavior EmptyBehavior) ([]s2.Region, error) { var regions []s2.Region + if geomRepr.Empty() { + switch emptyBehavior { + case EmptyBehaviorOmit: + return nil, nil + case EmptyBehaviorError: + return nil, NewEmptyGeometryError() + default: + return nil, errors.Newf("programmer error: unknown behavior") + } + } switch repr := geomRepr.(type) { case *geom.Point: regions = []s2.Region{ @@ -446,22 +469,38 @@ func S2RegionsFromGeom(geomRepr geom.T) []s2.Region { } case *geom.GeometryCollection: for _, geom := range repr.Geoms() { - regions = append(regions, S2RegionsFromGeom(geom)...) + subRegions, err := S2RegionsFromGeom(geom, emptyBehavior) + if err != nil { + return nil, err + } + regions = append(regions, subRegions...) } case *geom.MultiPoint: for i := 0; i < repr.NumPoints(); i++ { - regions = append(regions, S2RegionsFromGeom(repr.Point(i))...) + subRegions, err := S2RegionsFromGeom(repr.Point(i), emptyBehavior) + if err != nil { + return nil, err + } + regions = append(regions, subRegions...) } case *geom.MultiLineString: for i := 0; i < repr.NumLineStrings(); i++ { - regions = append(regions, S2RegionsFromGeom(repr.LineString(i))...) + subRegions, err := S2RegionsFromGeom(repr.LineString(i), emptyBehavior) + if err != nil { + return nil, err + } + regions = append(regions, subRegions...) } case *geom.MultiPolygon: for i := 0; i < repr.NumPolygons(); i++ { - regions = append(regions, S2RegionsFromGeom(repr.Polygon(i))...) + subRegions, err := S2RegionsFromGeom(repr.Polygon(i), emptyBehavior) + if err != nil { + return nil, err + } + regions = append(regions, subRegions...) } } - return regions + return regions, nil } // diff --git a/pkg/geo/geo_test.go b/pkg/geo/geo_test.go index e3e4d0d51908..4cf21e09577d 100644 --- a/pkg/geo/geo_test.go +++ b/pkg/geo/geo_test.go @@ -421,12 +421,58 @@ func TestGeographyAsS2(t *testing.T) { g, err := ParseGeography(tc.wkt) require.NoError(t, err) - shapes, err := g.AsS2() + shapes, err := g.AsS2(EmptyBehaviorError) require.NoError(t, err) require.Equal(t, tc.expected, shapes) }) } + + // Test when things are empty. + emptyTestCases := []struct { + wkt string + expectedOmit []s2.Region + }{ + { + "GEOMETRYCOLLECTION ( LINESTRING EMPTY, MULTIPOINT((1.0 5.0), (3.0 4.0)) )", + []s2.Region{ + s2.PointFromLatLng(s2.LatLngFromDegrees(5.0, 1.0)), + s2.PointFromLatLng(s2.LatLngFromDegrees(4.0, 3.0)), + }, + }, + { + "GEOMETRYCOLLECTION EMPTY", + nil, + }, + { + "MULTILINESTRING (EMPTY, (1.0 2.0, 3.0 4.0))", + []s2.Region{ + s2.PolylineFromLatLngs([]s2.LatLng{ + s2.LatLngFromDegrees(2.0, 1.0), + s2.LatLngFromDegrees(4.0, 3.0), + }), + }, + }, + { + "MULTILINESTRING (EMPTY, EMPTY)", + nil, + }, + } + + for _, tc := range emptyTestCases { + t.Run(tc.wkt, func(t *testing.T) { + g, err := ParseGeography(tc.wkt) + require.NoError(t, err) + + _, err = g.AsS2(EmptyBehaviorError) + require.Error(t, err) + require.True(t, IsEmptyGeometryError(err)) + + shapes, err := g.AsS2(EmptyBehaviorOmit) + require.NoError(t, err) + require.Equal(t, tc.expectedOmit, shapes) + }) + } } func TestClipEWKBByRect(t *testing.T) { diff --git a/pkg/geo/geogfn/covers.go b/pkg/geo/geogfn/covers.go index 3bed52d8b801..8989ea0a9a54 100644 --- a/pkg/geo/geogfn/covers.go +++ b/pkg/geo/geogfn/covers.go @@ -48,12 +48,17 @@ func Covers(a *geo.Geography, b *geo.Geography) (bool, error) { // covers is the internal calculation for Covers. func covers(a *geo.Geography, b *geo.Geography) (bool, error) { - aRegions, err := a.AsS2() + // Ignore EMPTY regions in a. + aRegions, err := a.AsS2(geo.EmptyBehaviorOmit) if err != nil { return false, err } - bRegions, err := b.AsS2() + // If any of b is empty, we cannot cover it. Error and catch to return false. + bRegions, err := b.AsS2(geo.EmptyBehaviorError) if err != nil { + if geo.IsEmptyGeometryError(err) { + return false, nil + } return false, err } diff --git a/pkg/geo/geogfn/covers_test.go b/pkg/geo/geogfn/covers_test.go index a3e6237aa0cb..4a0470dea339 100644 --- a/pkg/geo/geogfn/covers_test.go +++ b/pkg/geo/geogfn/covers_test.go @@ -354,6 +354,30 @@ func TestCovers(t *testing.T) { "MULTIPOINT((0.5 0.5), (1.5 0.5))'", true, }, + { + "EMPTY GEOMETRYCOLLECTION does not cover itself", + "GEOMETRYCOLLECTION EMPTY", + "GEOMETRYCOLLECTION EMPTY", + false, + }, + { + "nothing covers an empty GEOMETRYCOLLECTION", + "POINT(1.0 1.0)", + "GEOMETRYCOLLECTION EMPTY", + false, + }, + { + "nothing covers a GEOMETRYCOLLECTION with an EMPTY element", + "POINT(1.0 1.0)", + "GEOMETRYCOLLECTION EMPTY", + false, + }, + { + "empty collection contains point which covers another", + "GEOMETRYCOLLECTION(LINESTRING EMPTY, POINT(1.0 2.0))", + "POINT(1.0 2.0)", + true, + }, } for _, tc := range testCases { diff --git a/pkg/geo/geogfn/distance.go b/pkg/geo/geogfn/distance.go index 0370d6ca2227..23a3ca1dadc7 100644 --- a/pkg/geo/geogfn/distance.go +++ b/pkg/geo/geogfn/distance.go @@ -22,6 +22,7 @@ import ( ) // Distance returns the distance between geographies a and b on a sphere or spheroid. +// Returns a geo.EmptyGeometryError if any of the Geographies are EMPTY. func Distance( a *geo.Geography, b *geo.Geography, useSphereOrSpheroid UseSphereOrSpheroid, ) (float64, error) { @@ -29,11 +30,11 @@ func Distance( return 0, geo.NewMismatchingSRIDsError(a, b) } - aRegions, err := a.AsS2() + aRegions, err := a.AsS2(geo.EmptyBehaviorError) if err != nil { return 0, err } - bRegions, err := b.AsS2() + bRegions, err := b.AsS2(geo.EmptyBehaviorError) if err != nil { return 0, err } diff --git a/pkg/geo/geogfn/distance_test.go b/pkg/geo/geogfn/distance_test.go index 986a8ae6f17e..f8ee0270dc5b 100644 --- a/pkg/geo/geogfn/distance_test.go +++ b/pkg/geo/geogfn/distance_test.go @@ -11,6 +11,7 @@ package geogfn import ( + "fmt" "math" "testing" @@ -270,4 +271,30 @@ func TestDistance(t *testing.T) { _, err := Distance(mismatchingSRIDGeographyA, mismatchingSRIDGeographyB, UseSpheroid) requireMismatchingSRIDError(t, err) }) + + t.Run("empty geographies always error", func(t *testing.T) { + for _, tc := range []struct { + a string + b string + }{ + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION EMPTY"}, + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION (POINT(1.0 1.0), LINESTRING EMPTY)"}, + {"POINT(1.0 1.0)", "GEOMETRYCOLLECTION (POINT(1.0 1.0), LINESTRING EMPTY)"}, + } { + for _, useSphereOrSpheroid := range []UseSphereOrSpheroid{ + UseSphere, + UseSpheroid, + } { + t.Run(fmt.Sprintf("Distance(%s,%s),spheroid=%t", tc.a, tc.b, useSphereOrSpheroid), func(t *testing.T) { + a, err := geo.ParseGeography(tc.a) + require.NoError(t, err) + b, err := geo.ParseGeography(tc.b) + require.NoError(t, err) + _, err = Distance(a, b, useSphereOrSpheroid) + require.Error(t, err) + require.True(t, geo.IsEmptyGeometryError(err)) + }) + } + } + }) } diff --git a/pkg/geo/geogfn/dwithin.go b/pkg/geo/geogfn/dwithin.go index 9d70dd094910..74611da37110 100644 --- a/pkg/geo/geogfn/dwithin.go +++ b/pkg/geo/geogfn/dwithin.go @@ -19,6 +19,7 @@ import ( ) // DWithin returns whether a is within distance d of b, i.e. Distance(a, b) <= d. +// If A or B contains empty Geography objects, this will return false. func DWithin( a *geo.Geography, b *geo.Geography, distance float64, useSphereOrSpheroid UseSphereOrSpheroid, ) (bool, error) { @@ -29,12 +30,18 @@ func DWithin( return false, errors.Newf("dwithin distance cannot be less than zero") } - aRegions, err := a.AsS2() + aRegions, err := a.AsS2(geo.EmptyBehaviorError) if err != nil { + if geo.IsEmptyGeometryError(err) { + return false, nil + } return false, err } - bRegions, err := b.AsS2() + bRegions, err := b.AsS2(geo.EmptyBehaviorError) if err != nil { + if geo.IsEmptyGeometryError(err) { + return false, nil + } return false, err } spheroid := geographiclib.WGS84Spheroid diff --git a/pkg/geo/geogfn/dwithin_test.go b/pkg/geo/geogfn/dwithin_test.go index a6f95813cbc7..fecd9766fb48 100644 --- a/pkg/geo/geogfn/dwithin_test.go +++ b/pkg/geo/geogfn/dwithin_test.go @@ -106,4 +106,30 @@ func TestDWithin(t *testing.T) { _, err := DWithin(geo.MustParseGeography("POINT(1.0 2.0)"), geo.MustParseGeography("POINT(3.0 4.0)"), -0.01, UseSpheroid) require.Error(t, err) }) + + t.Run("empty geographies are never dwithin each other", func(t *testing.T) { + for _, tc := range []struct { + a string + b string + }{ + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION EMPTY"}, + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION (POINT(1.0 1.0), LINESTRING EMPTY)"}, + {"POINT(1.0 1.0)", "GEOMETRYCOLLECTION (POINT(1.0 1.0), LINESTRING EMPTY)"}, // This case errors (in a bad way) in PostGIS. + } { + for _, useSphereOrSpheroid := range []UseSphereOrSpheroid{ + UseSphere, + UseSpheroid, + } { + t.Run(fmt.Sprintf("DWithin(%s,%s),spheroid=%t", tc.a, tc.b, useSphereOrSpheroid), func(t *testing.T) { + a, err := geo.ParseGeography(tc.a) + require.NoError(t, err) + b, err := geo.ParseGeography(tc.b) + require.NoError(t, err) + dwithin, err := DWithin(a, b, 0, useSphereOrSpheroid) + require.NoError(t, err) + require.False(t, dwithin) + }) + } + } + }) } diff --git a/pkg/geo/geogfn/intersects.go b/pkg/geo/geogfn/intersects.go index 11cd9dc12c19..a9f8216b69ba 100644 --- a/pkg/geo/geogfn/intersects.go +++ b/pkg/geo/geogfn/intersects.go @@ -25,11 +25,11 @@ func Intersects(a *geo.Geography, b *geo.Geography) (bool, error) { return false, geo.NewMismatchingSRIDsError(a, b) } - aRegions, err := a.AsS2() + aRegions, err := a.AsS2(geo.EmptyBehaviorOmit) if err != nil { return false, err } - bRegions, err := b.AsS2() + bRegions, err := b.AsS2(geo.EmptyBehaviorOmit) if err != nil { return false, err } diff --git a/pkg/geo/geogfn/intersects_test.go b/pkg/geo/geogfn/intersects_test.go index 016c13618026..072e6ad3e42a 100644 --- a/pkg/geo/geogfn/intersects_test.go +++ b/pkg/geo/geogfn/intersects_test.go @@ -258,6 +258,24 @@ func TestIntersects(t *testing.T) { "MULTIPOLYGON (((0.0 0.0, 1.0 0.0, 1.0 1.0, 0.0 1.0, 0.0 0.0)))", false, }, + { + "GEOMETRYCOLLECTION EMPTY do not intersect with each other", + "GEOMETRYCOLLECTION EMPTY", + "GEOMETRYCOLLECTION EMPTY", + false, + }, + { + "GEOMETRYCOLLECTION EMPTY do not intersect with a point", + "POINT(1.0 2.0)", + "GEOMETRYCOLLECTION EMPTY", + false, + }, + { + "GEOMETRYCOLLECTION EMPTY and POINT intersect", + "POINT(1.0 2.0)", + "GEOMETRYCOLLECTION (LINESTRING EMPTY, POINT(1.0 2.0))", + true, + }, } for _, tc := range testCases { diff --git a/pkg/geo/geogfn/unary_operators.go b/pkg/geo/geogfn/unary_operators.go index 86b14a46ceac..e5ce3ccf7c05 100644 --- a/pkg/geo/geogfn/unary_operators.go +++ b/pkg/geo/geogfn/unary_operators.go @@ -20,7 +20,7 @@ import ( // Area returns the area of a given Geography. func Area(g *geo.Geography, useSphereOrSpheroid UseSphereOrSpheroid) (float64, error) { - regions, err := g.AsS2() + regions, err := g.AsS2(geo.EmptyBehaviorOmit) if err != nil { return 0, err } @@ -63,7 +63,11 @@ func Perimeter(g *geo.Geography, useSphereOrSpheroid UseSphereOrSpheroid) (float default: return 0, nil } - return length(geo.S2RegionsFromGeom(gt), useSphereOrSpheroid) + regions, err := geo.S2RegionsFromGeom(gt, geo.EmptyBehaviorOmit) + if err != nil { + return 0, err + } + return length(regions, useSphereOrSpheroid) } // Length returns length of a given Geography. @@ -79,7 +83,11 @@ func Length(g *geo.Geography, useSphereOrSpheroid UseSphereOrSpheroid) (float64, default: return 0, nil } - return length(geo.S2RegionsFromGeom(gt), useSphereOrSpheroid) + regions, err := geo.S2RegionsFromGeom(gt, geo.EmptyBehaviorOmit) + if err != nil { + return 0, err + } + return length(regions, useSphereOrSpheroid) } // length returns the sum of the lengtsh and perimeters in the shapes of the Geography. diff --git a/pkg/geo/geogfn/unary_operators_test.go b/pkg/geo/geogfn/unary_operators_test.go index 9090c790c3ae..6149ad6762cb 100644 --- a/pkg/geo/geogfn/unary_operators_test.go +++ b/pkg/geo/geogfn/unary_operators_test.go @@ -99,6 +99,32 @@ var unaryOperatorTestCases = []struct { expectedPerimeter: 9632838.874863794, }, }, + { + wkt: "GEOMETRYCOLLECTION (MULTIPOINT EMPTY, POINT (40 10),LINESTRING (10 10, 20 20, 10 40),POLYGON ((40 40, 20 45, 45 30, 40 40)))", + sphere: unaryOperatorExpectedResult{ + expectedArea: 691570576619.521, + expectedLength: 9637039.459995955, + expectedPerimeter: 9637039.459995955, + }, + spheroid: unaryOperatorExpectedResult{ + expectedArea: 691638769184.1753, + expectedLength: 9632838.874863794, + expectedPerimeter: 9632838.874863794, + }, + }, + { + wkt: "GEOMETRYCOLLECTION EMPTY", + sphere: unaryOperatorExpectedResult{ + expectedArea: 0, + expectedLength: 0, + expectedPerimeter: 0, + }, + spheroid: unaryOperatorExpectedResult{ + expectedArea: 0, + expectedLength: 0, + expectedPerimeter: 0, + }, + }, } func TestArea(t *testing.T) { diff --git a/pkg/geo/geoindex/s2_geography_index.go b/pkg/geo/geoindex/s2_geography_index.go index 67f88ed77874..77d67181e190 100644 --- a/pkg/geo/geoindex/s2_geography_index.go +++ b/pkg/geo/geoindex/s2_geography_index.go @@ -51,7 +51,7 @@ func DefaultGeographyIndexConfig() *Config { // InvertedIndexKeys implements the GeographyIndex interface. func (i *s2GeographyIndex) InvertedIndexKeys(c context.Context, g *geo.Geography) ([]Key, error) { - r, err := g.AsS2() + r, err := g.AsS2(geo.EmptyBehaviorOmit) if err != nil { return nil, err } @@ -60,7 +60,7 @@ func (i *s2GeographyIndex) InvertedIndexKeys(c context.Context, g *geo.Geography // Covers implements the GeographyIndex interface. func (i *s2GeographyIndex) Covers(c context.Context, g *geo.Geography) (UnionKeySpans, error) { - r, err := g.AsS2() + r, err := g.AsS2(geo.EmptyBehaviorOmit) if err != nil { return nil, err } @@ -69,7 +69,7 @@ func (i *s2GeographyIndex) Covers(c context.Context, g *geo.Geography) (UnionKey // CoveredBy implements the GeographyIndex interface. func (i *s2GeographyIndex) CoveredBy(c context.Context, g *geo.Geography) (RPKeyExpr, error) { - r, err := g.AsS2() + r, err := g.AsS2(geo.EmptyBehaviorOmit) if err != nil { return nil, err } @@ -78,7 +78,7 @@ func (i *s2GeographyIndex) CoveredBy(c context.Context, g *geo.Geography) (RPKey // Intersects implements the GeographyIndex interface. func (i *s2GeographyIndex) Intersects(c context.Context, g *geo.Geography) (UnionKeySpans, error) { - r, err := g.AsS2() + r, err := g.AsS2(geo.EmptyBehaviorOmit) if err != nil { return nil, err } @@ -86,7 +86,7 @@ func (i *s2GeographyIndex) Intersects(c context.Context, g *geo.Geography) (Unio } func (i *s2GeographyIndex) testingInnerCovering(g *geo.Geography) s2.CellUnion { - r, _ := g.AsS2() + r, _ := g.AsS2(geo.EmptyBehaviorOmit) if r == nil { return nil } diff --git a/pkg/geo/geomfn/distance.go b/pkg/geo/geomfn/distance.go index 226b75236a5b..81faa604810e 100644 --- a/pkg/geo/geomfn/distance.go +++ b/pkg/geo/geomfn/distance.go @@ -21,11 +21,12 @@ import ( ) // MinDistance returns the minimum distance between geometries A and B. +// This returns a geo.EmptyGeometryError if either A or B is EMPTY. func MinDistance(a *geo.Geometry, b *geo.Geometry) (float64, error) { if a.SRID() != b.SRID() { return 0, geo.NewMismatchingSRIDsError(a, b) } - return minDistanceInternal(a, b, 0) + return minDistanceInternal(a, b, 0, geo.EmptyBehaviorOmit) } // MaxDistance returns the maximum distance across every pair of points comprising @@ -34,7 +35,7 @@ func MaxDistance(a *geo.Geometry, b *geo.Geometry) (float64, error) { if a.SRID() != b.SRID() { return 0, geo.NewMismatchingSRIDsError(a, b) } - return maxDistanceInternal(a, b, math.MaxFloat64) + return maxDistanceInternal(a, b, math.MaxFloat64, geo.EmptyBehaviorOmit) } // DWithin determines if any part of geometry A is within D units of geometry B. @@ -45,8 +46,12 @@ func DWithin(a *geo.Geometry, b *geo.Geometry, d float64) (bool, error) { if d < 0 { return false, errors.Newf("dwithin distance cannot be less than zero") } - dist, err := minDistanceInternal(a, b, d) + dist, err := minDistanceInternal(a, b, d, geo.EmptyBehaviorError) if err != nil { + // In case of any empty geometries return false. + if geo.IsEmptyGeometryError(err) { + return false, nil + } return false, err } return dist <= d, nil @@ -61,8 +66,12 @@ func DFullyWithin(a *geo.Geometry, b *geo.Geometry, d float64) (bool, error) { if d < 0 { return false, errors.Newf("dwithin distance cannot be less than zero") } - dist, err := maxDistanceInternal(a, b, d) + dist, err := maxDistanceInternal(a, b, d, geo.EmptyBehaviorError) if err != nil { + // In case of any empty geometries return false. + if geo.IsEmptyGeometryError(err) { + return false, nil + } return false, err } return dist <= d, nil @@ -71,33 +80,45 @@ func DFullyWithin(a *geo.Geometry, b *geo.Geometry, d float64) (bool, error) { // maxDistanceInternal finds the maximum distance between two geometries. // We can re-use the same algorithm as min-distance, allowing skips of checks that involve // the interiors or intersections as those will always be less then the maximum min-distance. -func maxDistanceInternal(a *geo.Geometry, b *geo.Geometry, stopAfterGT float64) (float64, error) { +func maxDistanceInternal( + a *geo.Geometry, b *geo.Geometry, stopAfterGT float64, emptyBehavior geo.EmptyBehavior, +) (float64, error) { u := newGeomMaxDistanceUpdater(stopAfterGT) c := &geomDistanceCalculator{updater: u} - return distanceInternal(a, b, c) + return distanceInternal(a, b, c, emptyBehavior) } // minDistanceInternal finds the minimum distance between two geometries. // This implementation is done in-house, as compared to using GEOS. -func minDistanceInternal(a *geo.Geometry, b *geo.Geometry, stopAfterLE float64) (float64, error) { +func minDistanceInternal( + a *geo.Geometry, b *geo.Geometry, stopAfterLE float64, emptyBehavior geo.EmptyBehavior, +) (float64, error) { u := newGeomMinDistanceUpdater(stopAfterLE) c := &geomDistanceCalculator{updater: u} - return distanceInternal(a, b, c) + return distanceInternal(a, b, c, emptyBehavior) } // distanceInternal calculates the distance between two geometries using // the DistanceCalculator operator. +// If there are any EMPTY Geometry objects, they will be ignored. It will return an +// EmptyGeometryError if A or B contains only EMPTY geometries, even if emptyBehavior +// is set to EmptyBehaviorOmit. func distanceInternal( - a *geo.Geometry, b *geo.Geometry, c geodist.DistanceCalculator, + a *geo.Geometry, b *geo.Geometry, c geodist.DistanceCalculator, emptyBehavior geo.EmptyBehavior, ) (float64, error) { - aGeoms, err := flattenGeometry(a) + aGeoms, err := flattenGeometry(a, emptyBehavior) if err != nil { return 0, err } - bGeoms, err := flattenGeometry(b) + bGeoms, err := flattenGeometry(b, emptyBehavior) if err != nil { return 0, err } + // If either side has no geoms, then we error out. + if len(aGeoms) == 0 || len(bGeoms) == 0 { + return 0, geo.NewEmptyGeometryError() + } + for _, aGeom := range aGeoms { aGeodist, err := geomToGeodist(aGeom) if err != nil { diff --git a/pkg/geo/geomfn/distance_test.go b/pkg/geo/geomfn/distance_test.go index 72a1fcc92ed5..abc595ddafb6 100644 --- a/pkg/geo/geomfn/distance_test.go +++ b/pkg/geo/geomfn/distance_test.go @@ -195,6 +195,41 @@ var distanceTestCases = []struct { 0, 50, }, + { + "GEOMETRYCOLLECTION (POINT, EMPTY) with POINT", + "GEOMETRYCOLLECTION ( POINT(1.0 2.0), LINESTRING EMPTY )", + "POINT(1.0 2.0)", + 0, + 0, + }, + { + "GEOMETRYCOLLECTION (POINT, EMPTY) with DIFFERENT POINT", + "GEOMETRYCOLLECTION ( POINT(1.0 2.0), LINESTRING EMPTY )", + "POINT(1.0 3.0)", + 1, + 1, + }, +} + +// TODO(otan): delete after https://github.com/cockroachdb/cockroach/issues/49209 +var knownGEOSPanics = map[string]struct{}{ + "GEOMETRYCOLLECTION (POINT, EMPTY) with POINT": {}, + "GEOMETRYCOLLECTION (POINT, EMPTY) with DIFFERENT POINT": {}, +} + +var falseDWithinTestCases = map[string]struct{}{ + "GEOMETRYCOLLECTION (POINT, EMPTY) with POINT": {}, + "GEOMETRYCOLLECTION (POINT, EMPTY) with DIFFERENT POINT": {}, +} + +var emptyDistanceTestCases = []struct { + a string + b string +}{ + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION EMPTY"}, + {"GEOMETRYCOLLECTION EMPTY", "GEOMETRYCOLLECTION (LINESTRING EMPTY)"}, + {"GEOMETRYCOLLECTION EMPTY", "POINT(1.0 1.0)"}, + {"POINT(1.0 1.0)", "GEOMETRYCOLLECTION EMPTY"}, } func TestMinDistance(t *testing.T) { @@ -215,19 +250,35 @@ func TestMinDistance(t *testing.T) { require.Equal(t, tc.expectedMinDistance, ret) // Check distance roughly the same as GEOS. - ret, err = geos.MinDistance(a.EWKB(), b.EWKB()) - require.NoError(t, err) - require.LessOrEqualf( - t, - math.Abs(tc.expectedMinDistance-ret), - 0.0000001, // GEOS and PostGIS/CRDB can return results close by. - "expected distance within %f, GEOS returns %f", - tc.expectedMinDistance, - ret, - ) + if _, panicsInGEOS := knownGEOSPanics[tc.desc]; !panicsInGEOS { + ret, err = geos.MinDistance(a.EWKB(), b.EWKB()) + require.NoError(t, err) + require.LessOrEqualf( + t, + math.Abs(tc.expectedMinDistance-ret), + 0.0000001, // GEOS and PostGIS/CRDB can return results close by. + "expected distance within %f, GEOS returns %f", + tc.expectedMinDistance, + ret, + ) + } }) } + t.Run("errors for EMPTY geometries", func(t *testing.T) { + for _, tc := range emptyDistanceTestCases { + t.Run(fmt.Sprintf("%s to %s", tc.a, tc.b), func(t *testing.T) { + a, err := geo.ParseGeometry(tc.a) + require.NoError(t, err) + b, err := geo.ParseGeometry(tc.b) + require.NoError(t, err) + _, err = MinDistance(a, b) + require.Error(t, err) + require.True(t, geo.IsEmptyGeometryError(err)) + }) + } + }) + t.Run("errors if SRIDs mismatch", func(t *testing.T) { _, err := MinDistance(mismatchingSRIDGeometryA, mismatchingSRIDGeometryB) requireMismatchingSRIDError(t, err) @@ -267,6 +318,12 @@ func TestDWithin(t *testing.T) { b, err := geo.ParseGeometry(tc.b) require.NoError(t, err) + // empty geometries should always return false. + expected := true + if _, ok := falseDWithinTestCases[tc.desc]; ok { + expected = false + } + for _, val := range []float64{ tc.expectedMinDistance, tc.expectedMinDistance + 0.1, @@ -276,11 +333,11 @@ func TestDWithin(t *testing.T) { t.Run(fmt.Sprintf("dwithin:%f", val), func(t *testing.T) { dwithin, err := DWithin(a, b, val) require.NoError(t, err) - require.True(t, dwithin) + require.Equal(t, expected, dwithin) dwithin, err = DWithin(a, b, val) require.NoError(t, err) - require.True(t, dwithin) + require.Equal(t, expected, dwithin) }) } @@ -304,6 +361,20 @@ func TestDWithin(t *testing.T) { }) } + t.Run("returns false for EMPTY geometries", func(t *testing.T) { + for _, tc := range emptyDistanceTestCases { + t.Run(fmt.Sprintf("%s to %s", tc.a, tc.b), func(t *testing.T) { + a, err := geo.ParseGeometry(tc.a) + require.NoError(t, err) + b, err := geo.ParseGeometry(tc.b) + require.NoError(t, err) + dwithin, err := DWithin(a, b, 0) + require.NoError(t, err) + require.False(t, dwithin) + }) + } + }) + t.Run("errors if SRIDs mismatch", func(t *testing.T) { _, err := MinDistance(mismatchingSRIDGeometryA, mismatchingSRIDGeometryB) requireMismatchingSRIDError(t, err) @@ -323,6 +394,12 @@ func TestDFullyWithin(t *testing.T) { b, err := geo.ParseGeometry(tc.b) require.NoError(t, err) + // empty geometries should always return false. + expected := true + if _, ok := falseDWithinTestCases[tc.desc]; ok { + expected = false + } + for _, val := range []float64{ tc.expectedMaxDistance, tc.expectedMaxDistance + 0.1, @@ -330,13 +407,13 @@ func TestDFullyWithin(t *testing.T) { tc.expectedMaxDistance * 2, } { t.Run(fmt.Sprintf("dfullywithin:%f", val), func(t *testing.T) { - dwithax, err := DFullyWithin(a, b, val) + dfullywithin, err := DFullyWithin(a, b, val) require.NoError(t, err) - require.True(t, dwithax) + require.Equal(t, expected, dfullywithin) - dwithax, err = DFullyWithin(a, b, val) + dfullywithin, err = DFullyWithin(a, b, val) require.NoError(t, err) - require.True(t, dwithax) + require.Equal(t, expected, dfullywithin) }) } @@ -347,13 +424,13 @@ func TestDFullyWithin(t *testing.T) { } { if val > 0 { t.Run(fmt.Sprintf("dfullywithin:%f", val), func(t *testing.T) { - dwithin, err := DFullyWithin(a, b, val) + dfullywithin, err := DFullyWithin(a, b, val) require.NoError(t, err) - require.False(t, dwithin) + require.False(t, dfullywithin) - dwithin, err = DFullyWithin(a, b, val) + dfullywithin, err = DFullyWithin(a, b, val) require.NoError(t, err) - require.False(t, dwithin) + require.False(t, dfullywithin) }) } } diff --git a/pkg/geo/geomfn/geomfn.go b/pkg/geo/geomfn/geomfn.go index dc3f2833efd0..8c7257bcd012 100644 --- a/pkg/geo/geomfn/geomfn.go +++ b/pkg/geo/geomfn/geomfn.go @@ -18,16 +18,26 @@ import ( ) // flattenGeometry flattens a geo.Geometry object. -func flattenGeometry(g *geo.Geometry) ([]geom.T, error) { +func flattenGeometry(g *geo.Geometry, emptyBehavior geo.EmptyBehavior) ([]geom.T, error) { f, err := g.AsGeomT() if err != nil { return nil, err } - return flattenGeomT(f) + return flattenGeomT(f, emptyBehavior) } // flattenGeomT decomposes geom.T collections to individual geom.T components. -func flattenGeomT(g geom.T) ([]geom.T, error) { +func flattenGeomT(g geom.T, emptyBehavior geo.EmptyBehavior) ([]geom.T, error) { + if g.Empty() { + switch emptyBehavior { + case geo.EmptyBehaviorOmit: + return nil, nil + case geo.EmptyBehaviorError: + return nil, geo.NewEmptyGeometryError() + default: + return nil, errors.Newf("programmer error: unknown behavior") + } + } switch g := g.(type) { case *geom.Point: return []geom.T{g}, nil @@ -54,9 +64,19 @@ func flattenGeomT(g geom.T) ([]geom.T, error) { } return ret, nil case *geom.GeometryCollection: - ret := make([]geom.T, g.NumGeoms()) - for i := 0; i < g.NumGeoms(); i++ { - ret[i] = g.Geom(i) + ret := make([]geom.T, 0, g.NumGeoms()) + for _, subG := range g.Geoms() { + if subG.Empty() { + switch emptyBehavior { + case geo.EmptyBehaviorOmit: + continue + case geo.EmptyBehaviorError: + return nil, geo.NewEmptyGeometryError() + default: + return nil, errors.Newf("programmer error: unknown behavior") + } + } + ret = append(ret, subG) } return ret, nil } diff --git a/pkg/sql/logictest/testdata/logic_test/geospatial b/pkg/sql/logictest/testdata/logic_test/geospatial index e86cafbdbb10..3edcce86f3f9 100644 --- a/pkg/sql/logictest/testdata/logic_test/geospatial +++ b/pkg/sql/logictest/testdata/logic_test/geospatial @@ -299,7 +299,10 @@ INSERT INTO geom_operators_test VALUES ('Point middle of Right Square', 'POINT(0.5 0.5)'), ('Square overlapping left and right square', 'POLYGON((-0.1 0.0, 1.0 0.0, 1.0 1.0, -0.1 1.0, -0.1 0.0))'), ('Line going through left and right square', 'LINESTRING(-0.5 0.5, 0.5 0.5)'), - ('Faraway point', 'POINT(5.0 5.0)') + ('Faraway point', 'POINT(5.0 5.0)'), + ('Empty LineString', 'LINESTRING EMPTY'), + ('Empty GeometryCollection', 'GEOMETRYCOLLECTION EMPTY') + -- ('Partially Empty GeometryCollection', 'GEOMETRYCOLLECTION ( LINESTRING EMPTY, POINT (0.0 0.0) )'): omitted, see: #49209 # Unary operators query TRRR @@ -311,6 +314,8 @@ SELECT FROM geom_operators_test a ORDER BY a.dsc ---- +Empty GeometryCollection 0 0 0 +Empty LineString 0 0 0 Faraway point 0 0 0 Line going through left and right square 0 1 0 NULL NULL NULL NULL @@ -331,6 +336,28 @@ FROM geom_operators_test a JOIN geom_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection NULL NULL +Empty GeometryCollection Empty LineString NULL NULL +Empty GeometryCollection Faraway point NULL NULL +Empty GeometryCollection Line going through left and right square NULL NULL +Empty GeometryCollection NULL NULL NULL +Empty GeometryCollection Point middle of Left Square NULL NULL +Empty GeometryCollection Point middle of Right Square NULL NULL +Empty GeometryCollection Square (left) NULL NULL +Empty GeometryCollection Square (right) NULL NULL +Empty GeometryCollection Square overlapping left and right square NULL NULL +Empty LineString Empty GeometryCollection NULL NULL +Empty LineString Empty LineString NULL NULL +Empty LineString Faraway point NULL NULL +Empty LineString Line going through left and right square NULL NULL +Empty LineString NULL NULL NULL +Empty LineString Point middle of Left Square NULL NULL +Empty LineString Point middle of Right Square NULL NULL +Empty LineString Square (left) NULL NULL +Empty LineString Square (right) NULL NULL +Empty LineString Square overlapping left and right square NULL NULL +Faraway point Empty GeometryCollection NULL NULL +Faraway point Empty LineString NULL NULL Faraway point Faraway point 0 0 Faraway point Line going through left and right square 6.36396103067893 7.10633520177595 Faraway point NULL NULL NULL @@ -339,6 +366,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) 6.40312423743285 7.81024967590665 Faraway point Square (right) 5.65685424949238 7.07106781186548 Faraway point Square overlapping left and right square 5.65685424949238 7.14212853426764 +Line going through left and right square Empty GeometryCollection NULL NULL +Line going through left and right square Empty LineString NULL NULL Line going through left and right square Faraway point 6.36396103067893 7.10633520177595 Line going through left and right square Line going through left and right square 0 1 Line going through left and right square NULL NULL NULL @@ -347,6 +376,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) 0 1.58113883008419 Line going through left and right square Square (right) 0 1.58113883008419 Line going through left and right square Square overlapping left and right square 0 1.58113883008419 +NULL Empty GeometryCollection NULL NULL +NULL Empty LineString NULL NULL NULL Faraway point NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL @@ -355,6 +386,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL Square (right) NULL NULL NULL Square overlapping left and right square NULL NULL +Point middle of Left Square Empty GeometryCollection NULL NULL +Point middle of Left Square Empty LineString NULL NULL Point middle of Left Square Faraway point 7.10633520177595 7.10633520177595 Point middle of Left Square Line going through left and right square 0 1 Point middle of Left Square NULL NULL NULL @@ -363,6 +396,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) 0 0.707106781186548 Point middle of Left Square Square (right) 0.5 1.58113883008419 Point middle of Left Square Square overlapping left and right square 0.4 1.58113883008419 +Point middle of Right Square Empty GeometryCollection NULL NULL +Point middle of Right Square Empty LineString NULL NULL Point middle of Right Square Faraway point 6.36396103067893 6.36396103067893 Point middle of Right Square Line going through left and right square 0 1 Point middle of Right Square NULL NULL NULL @@ -371,6 +406,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) 0.5 1.58113883008419 Point middle of Right Square Square (right) 0 0.707106781186548 Point middle of Right Square Square overlapping left and right square 0 0.781024967590665 +Square (left) Empty GeometryCollection NULL NULL +Square (left) Empty LineString NULL NULL Square (left) Faraway point 6.40312423743285 7.81024967590665 Square (left) Line going through left and right square 0 1.58113883008419 Square (left) NULL NULL NULL @@ -379,6 +416,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) 0 1.4142135623731 Square (left) Square (right) 0 2.23606797749979 Square (left) Square overlapping left and right square 0 2.23606797749979 +Square (right) Empty GeometryCollection NULL NULL +Square (right) Empty LineString NULL NULL Square (right) Faraway point 5.65685424949238 7.07106781186548 Square (right) Line going through left and right square 0 1.58113883008419 Square (right) NULL NULL NULL @@ -387,6 +426,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) 0 2.23606797749979 Square (right) Square (right) 0 1.4142135623731 Square (right) Square overlapping left and right square 0 1.48660687473185 +Square overlapping left and right square Empty GeometryCollection NULL NULL +Square overlapping left and right square Empty LineString NULL NULL Square overlapping left and right square Faraway point 5.65685424949238 7.14212853426764 Square overlapping left and right square Line going through left and right square 0 1.58113883008419 Square overlapping left and right square NULL NULL NULL @@ -415,6 +456,28 @@ FROM geom_operators_test a JOIN geom_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection false false false false false true false false false false +Empty GeometryCollection Empty LineString false false false false false true false false false false +Empty GeometryCollection Faraway point false false false false false false false false false false +Empty GeometryCollection Line going through left and right square false false false false false false false false false false +Empty GeometryCollection NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +Empty GeometryCollection Point middle of Left Square false false false false false false false false false false +Empty GeometryCollection Point middle of Right Square false false false false false false false false false false +Empty GeometryCollection Square (left) false false false false false false false false false false +Empty GeometryCollection Square (right) false false false false false false false false false false +Empty GeometryCollection Square overlapping left and right square false false false false false false false false false false +Empty LineString Empty GeometryCollection false false false false false true false false false false +Empty LineString Empty LineString false false false false false true false false false false +Empty LineString Faraway point false false false false false false false false false false +Empty LineString Line going through left and right square false false false false false false false false false false +Empty LineString NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +Empty LineString Point middle of Left Square false false false false false false false false false false +Empty LineString Point middle of Right Square false false false false false false false false false false +Empty LineString Square (left) false false false false false false false false false false +Empty LineString Square (right) false false false false false false false false false false +Empty LineString Square overlapping left and right square false false false false false false false false false false +Faraway point Empty GeometryCollection false false false false false false false false false false +Faraway point Empty LineString false false false false false false false false false false Faraway point Faraway point true true true true false true true false false true Faraway point Line going through left and right square false false false false false false false false false false Faraway point NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -423,6 +486,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) false false false false false false false false false false Faraway point Square (right) false false false false false false false false false false Faraway point Square overlapping left and right square false false false false false false false false false false +Line going through left and right square Empty GeometryCollection false false false false false false false false false false +Line going through left and right square Empty LineString false false false false false false false false false false Line going through left and right square Faraway point false false false false false false false false false false Line going through left and right square Line going through left and right square true true true false false true true false false true Line going through left and right square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -431,6 +496,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) false false false false true false true false false false Line going through left and right square Square (right) false false false false true false true false false false Line going through left and right square Square overlapping left and right square false false false false true false true false false false +NULL Empty GeometryCollection NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +NULL Empty LineString NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL Faraway point NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -439,6 +506,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL Square (right) NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL Square overlapping left and right square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL +Point middle of Left Square Empty GeometryCollection false false false false false false false false false false +Point middle of Left Square Empty LineString false false false false false false false false false false Point middle of Left Square Faraway point false false false false false false false false false false Point middle of Left Square Line going through left and right square false true false false false false true false true false Point middle of Left Square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -447,6 +516,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) false true false false false false true false false true Point middle of Left Square Square (right) false false false false false false false false false false Point middle of Left Square Square overlapping left and right square false false false false false false false false false false +Point middle of Right Square Empty GeometryCollection false false false false false false false false false false +Point middle of Right Square Empty LineString false false false false false false false false false false Point middle of Right Square Faraway point false false false false false false false false false false Point middle of Right Square Line going through left and right square false true false false false false true false true false Point middle of Right Square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -455,6 +526,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) false false false false false false false false false false Point middle of Right Square Square (right) false true false false false false true false false true Point middle of Right Square Square overlapping left and right square false true false false false false true false false true +Square (left) Empty GeometryCollection false false false false false false false false false false +Square (left) Empty LineString false false false false false false false false false false Square (left) Faraway point false false false false false false false false false false Square (left) Line going through left and right square false false false false true false true false false false Square (left) NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -463,6 +536,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) true true true false false true true false false true Square (left) Square (right) false false false false false false true false true false Square (left) Square overlapping left and right square false false false false false false true true false false +Square (right) Empty GeometryCollection false false false false false false false false false false +Square (right) Empty LineString false false false false false false false false false false Square (right) Faraway point false false false false false false false false false false Square (right) Line going through left and right square false false false false true false true false false false Square (right) NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -471,6 +546,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) false false false false false false true false true false Square (right) Square (right) true true true false false true true false false true Square (right) Square overlapping left and right square false true false false false false true false false true +Square overlapping left and right square Empty GeometryCollection false false false false false false false false false false +Square overlapping left and right square Empty LineString false false false false false false false false false false Square overlapping left and right square Faraway point false false false false false false false false false false Square overlapping left and right square Line going through left and right square false false false false true false true false false false Square overlapping left and right square NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -491,6 +568,28 @@ FROM geom_operators_test a JOIN geom_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection false false +Empty GeometryCollection Empty LineString false false +Empty GeometryCollection Faraway point false false +Empty GeometryCollection Line going through left and right square false false +Empty GeometryCollection NULL NULL NULL +Empty GeometryCollection Point middle of Left Square false false +Empty GeometryCollection Point middle of Right Square false false +Empty GeometryCollection Square (left) false false +Empty GeometryCollection Square (right) false false +Empty GeometryCollection Square overlapping left and right square false false +Empty LineString Empty GeometryCollection false false +Empty LineString Empty LineString false false +Empty LineString Faraway point false false +Empty LineString Line going through left and right square false false +Empty LineString NULL NULL NULL +Empty LineString Point middle of Left Square false false +Empty LineString Point middle of Right Square false false +Empty LineString Square (left) false false +Empty LineString Square (right) false false +Empty LineString Square overlapping left and right square false false +Faraway point Empty GeometryCollection false false +Faraway point Empty LineString false false Faraway point Faraway point true true Faraway point Line going through left and right square false false Faraway point NULL NULL NULL @@ -499,6 +598,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) false false Faraway point Square (right) false false Faraway point Square overlapping left and right square false false +Line going through left and right square Empty GeometryCollection false false +Line going through left and right square Empty LineString false false Line going through left and right square Faraway point false false Line going through left and right square Line going through left and right square true true Line going through left and right square NULL NULL NULL @@ -507,6 +608,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) true false Line going through left and right square Square (right) true false Line going through left and right square Square overlapping left and right square true false +NULL Empty GeometryCollection NULL NULL +NULL Empty LineString NULL NULL NULL Faraway point NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL @@ -515,6 +618,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL Square (right) NULL NULL NULL Square overlapping left and right square NULL NULL +Point middle of Left Square Empty GeometryCollection false false +Point middle of Left Square Empty LineString false false Point middle of Left Square Faraway point false false Point middle of Left Square Line going through left and right square true true Point middle of Left Square NULL NULL NULL @@ -523,6 +628,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) true true Point middle of Left Square Square (right) true false Point middle of Left Square Square overlapping left and right square true false +Point middle of Right Square Empty GeometryCollection false false +Point middle of Right Square Empty LineString false false Point middle of Right Square Faraway point false false Point middle of Right Square Line going through left and right square true true Point middle of Right Square NULL NULL NULL @@ -531,6 +638,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) true false Point middle of Right Square Square (right) true true Point middle of Right Square Square overlapping left and right square true true +Square (left) Empty GeometryCollection false false +Square (left) Empty LineString false false Square (left) Faraway point false false Square (left) Line going through left and right square true false Square (left) NULL NULL NULL @@ -539,6 +648,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) true false Square (left) Square (right) true false Square (left) Square overlapping left and right square true false +Square (right) Empty GeometryCollection false false +Square (right) Empty LineString false false Square (right) Faraway point false false Square (right) Line going through left and right square true false Square (right) NULL NULL NULL @@ -547,6 +658,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) true false Square (right) Square (right) true false Square (right) Square overlapping left and right square true false +Square overlapping left and right square Empty GeometryCollection false false +Square overlapping left and right square Empty LineString false false Square overlapping left and right square Faraway point false false Square overlapping left and right square Line going through left and right square true false Square overlapping left and right square NULL NULL NULL @@ -567,6 +680,28 @@ FROM geom_operators_test a JOIN geom_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection FFFFFFFF2 false +Empty GeometryCollection Empty LineString FFFFFFFF2 false +Empty GeometryCollection Faraway point FFFFFF0F2 false +Empty GeometryCollection Line going through left and right square FFFFFF102 false +Empty GeometryCollection NULL NULL NULL +Empty GeometryCollection Point middle of Left Square FFFFFF0F2 false +Empty GeometryCollection Point middle of Right Square FFFFFF0F2 false +Empty GeometryCollection Square (left) FFFFFF212 false +Empty GeometryCollection Square (right) FFFFFF212 false +Empty GeometryCollection Square overlapping left and right square FFFFFF212 false +Empty LineString Empty GeometryCollection FFFFFFFF2 false +Empty LineString Empty LineString FFFFFFFF2 false +Empty LineString Faraway point FFFFFF0F2 false +Empty LineString Line going through left and right square FFFFFF102 false +Empty LineString NULL NULL NULL +Empty LineString Point middle of Left Square FFFFFF0F2 false +Empty LineString Point middle of Right Square FFFFFF0F2 false +Empty LineString Square (left) FFFFFF212 false +Empty LineString Square (right) FFFFFF212 false +Empty LineString Square overlapping left and right square FFFFFF212 false +Faraway point Empty GeometryCollection FF0FFFFF2 false +Faraway point Empty LineString FF0FFFFF2 false Faraway point Faraway point 0FFFFFFF2 true Faraway point Line going through left and right square FF0FFF102 false Faraway point NULL NULL NULL @@ -575,6 +710,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) FF0FFF212 false Faraway point Square (right) FF0FFF212 false Faraway point Square overlapping left and right square FF0FFF212 false +Line going through left and right square Empty GeometryCollection FF1FF0FF2 false +Line going through left and right square Empty LineString FF1FF0FF2 false Line going through left and right square Faraway point FF1FF00F2 false Line going through left and right square Line going through left and right square 1FFF0FFF2 false Line going through left and right square NULL NULL NULL @@ -583,6 +720,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) 1010F0212 false Line going through left and right square Square (right) 1010F0212 false Line going through left and right square Square overlapping left and right square 1010F0212 false +NULL Empty GeometryCollection NULL NULL +NULL Empty LineString NULL NULL NULL Faraway point NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL @@ -591,6 +730,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL Square (right) NULL NULL NULL Square overlapping left and right square NULL NULL +Point middle of Left Square Empty GeometryCollection FF0FFFFF2 false +Point middle of Left Square Empty LineString FF0FFFFF2 false Point middle of Left Square Faraway point FF0FFF0F2 false Point middle of Left Square Line going through left and right square F0FFFF102 false Point middle of Left Square NULL NULL NULL @@ -599,6 +740,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) 0FFFFF212 false Point middle of Left Square Square (right) FF0FFF212 false Point middle of Left Square Square overlapping left and right square FF0FFF212 false +Point middle of Right Square Empty GeometryCollection FF0FFFFF2 false +Point middle of Right Square Empty LineString FF0FFFFF2 false Point middle of Right Square Faraway point FF0FFF0F2 false Point middle of Right Square Line going through left and right square F0FFFF102 false Point middle of Right Square NULL NULL NULL @@ -607,6 +750,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) FF0FFF212 false Point middle of Right Square Square (right) 0FFFFF212 false Point middle of Right Square Square overlapping left and right square 0FFFFF212 false +Square (left) Empty GeometryCollection FF2FF1FF2 false +Square (left) Empty LineString FF2FF1FF2 false Square (left) Faraway point FF2FF10F2 false Square (left) Line going through left and right square 1020F1102 false Square (left) NULL NULL NULL @@ -615,6 +760,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) 2FFF1FFF2 false Square (left) Square (right) FF2F11212 false Square (left) Square overlapping left and right square 212111212 false +Square (right) Empty GeometryCollection FF2FF1FF2 false +Square (right) Empty LineString FF2FF1FF2 false Square (right) Faraway point FF2FF10F2 false Square (right) Line going through left and right square 1020F1102 false Square (right) NULL NULL NULL @@ -623,6 +770,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) FF2F11212 false Square (right) Square (right) 2FFF1FFF2 false Square (right) Square overlapping left and right square 2FF11F212 false +Square overlapping left and right square Empty GeometryCollection FF2FF1FF2 false +Square overlapping left and right square Empty LineString FF2FF1FF2 false Square overlapping left and right square Faraway point FF2FF10F2 false Square overlapping left and right square Line going through left and right square 1020F1102 false Square overlapping left and right square NULL NULL NULL @@ -645,6 +794,8 @@ SELECT FROM geom_operators_test a ORDER BY a.dsc ---- +Empty GeometryCollection 0 0 0 NULL NULL NULL +Empty LineString 2 0 0 NULL 010200000000000000 NULL Faraway point 2 1 1 NULL 010100000000000000000014400000000000001440 NULL Line going through left and right square 2 2 1 NULL 010200000002000000000000000000E0BF000000000000E03F000000000000E03F000000000000E03F NULL NULL NULL NULL NULL NULL NULL NULL @@ -763,6 +914,8 @@ SELECT FROM geog_operators_test a ORDER BY a.dsc ---- +Empty GeometryCollection 0 0 0 0 0 0 0 0 0 +Empty LineString 0 0 0 0 0 0 0 0 0 Faraway point 0 0 0 0 0 0 0 0 0 Line going through left and right square 0 0 0 111315.280354463 111190.845659241 111315.280354463 0 0 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL @@ -782,6 +935,28 @@ FROM geog_operators_test a JOIN geog_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection NULL NULL NULL +Empty GeometryCollection Empty LineString NULL NULL NULL +Empty GeometryCollection Faraway point NULL NULL NULL +Empty GeometryCollection Line going through left and right square NULL NULL NULL +Empty GeometryCollection NULL NULL NULL NULL +Empty GeometryCollection Point middle of Left Square NULL NULL NULL +Empty GeometryCollection Point middle of Right Square NULL NULL NULL +Empty GeometryCollection Square (left) NULL NULL NULL +Empty GeometryCollection Square (right) NULL NULL NULL +Empty GeometryCollection Square overlapping left and right square NULL NULL NULL +Empty LineString Empty GeometryCollection NULL NULL NULL +Empty LineString Empty LineString NULL NULL NULL +Empty LineString Faraway point NULL NULL NULL +Empty LineString Line going through left and right square NULL NULL NULL +Empty LineString NULL NULL NULL NULL +Empty LineString Point middle of Left Square NULL NULL NULL +Empty LineString Point middle of Right Square NULL NULL NULL +Empty LineString Square (left) NULL NULL NULL +Empty LineString Square (right) NULL NULL NULL +Empty LineString Square overlapping left and right square NULL NULL NULL +Faraway point Empty GeometryCollection NULL NULL NULL +Faraway point Empty LineString NULL NULL NULL Faraway point Faraway point 0 0 0 Faraway point Line going through left and right square 705577.648328755 707142.162247235 705577.648328755 Faraway point NULL NULL NULL NULL @@ -790,6 +965,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) 710260.975395327 711311.966664979 710260.975395327 Faraway point Square (right) 627129.502610754 628519.033787529 627129.502610754 Faraway point Square overlapping left and right square 627129.502610754 628519.033787529 627129.502610754 +Line going through left and right square Empty GeometryCollection NULL NULL NULL +Line going through left and right square Empty LineString NULL NULL NULL Line going through left and right square Faraway point 705577.648328755 707142.162247235 705577.648328755 Line going through left and right square Line going through left and right square 0 0 0 Line going through left and right square NULL NULL NULL NULL @@ -798,6 +975,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) 0 0 0 Line going through left and right square Square (right) 0 0 0 Line going through left and right square Square overlapping left and right square 0 0 0 +NULL Empty GeometryCollection NULL NULL NULL +NULL Empty LineString NULL NULL NULL NULL Faraway point NULL NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL NULL NULL @@ -806,6 +985,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL NULL Square (right) NULL NULL NULL NULL Square overlapping left and right square NULL NULL NULL +Point middle of Left Square Empty GeometryCollection NULL NULL NULL +Point middle of Left Square Empty LineString NULL NULL NULL Point middle of Left Square Faraway point 788297.405265007 789521.780520966 788297.405265007 Point middle of Left Square Line going through left and right square 0 0 0 Point middle of Left Square NULL NULL NULL NULL @@ -814,6 +995,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) 0 0 0 Point middle of Left Square Square (right) 55657.6401772334 55595.4228296203 55657.6401772334 Point middle of Left Square Square overlapping left and right square 44526.1121572803 44476.338279173 44526.1121572803 +Point middle of Right Square Empty GeometryCollection NULL NULL NULL +Point middle of Right Square Empty LineString NULL NULL NULL Point middle of Right Square Faraway point 705577.648328755 707142.162247235 705577.648328755 Point middle of Right Square Line going through left and right square 0 0 0 Point middle of Right Square NULL NULL NULL NULL @@ -822,6 +1005,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) 55657.6401772334 55595.4228296203 55657.6401772334 Point middle of Right Square Square (right) 0 0 0 Point middle of Right Square Square overlapping left and right square 0 0 0 +Square (left) Empty GeometryCollection NULL NULL NULL +Square (left) Empty LineString NULL NULL NULL Square (left) Faraway point 710260.975395327 711311.966664979 710260.975395327 Square (left) Line going through left and right square 0 0 0 Square (left) NULL NULL NULL NULL @@ -830,6 +1015,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) 0 0 0 Square (left) Square (right) 0 0 0 Square (left) Square overlapping left and right square 0 0 0 +Square (right) Empty GeometryCollection NULL NULL NULL +Square (right) Empty LineString NULL NULL NULL Square (right) Faraway point 627129.502610754 628519.033787529 627129.502610754 Square (right) Line going through left and right square 0 0 0 Square (right) NULL NULL NULL NULL @@ -838,6 +1025,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) 0 0 0 Square (right) Square (right) 0 0 0 Square (right) Square overlapping left and right square 0 0 0 +Square overlapping left and right square Empty GeometryCollection NULL NULL NULL +Square overlapping left and right square Empty LineString NULL NULL NULL Square overlapping left and right square Faraway point 627129.502610754 628519.033787529 627129.502610754 Square overlapping left and right square Line going through left and right square 0 0 0 Square overlapping left and right square NULL NULL NULL NULL @@ -859,6 +1048,28 @@ FROM geog_operators_test a JOIN geog_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection false false false +Empty GeometryCollection Empty LineString false false false +Empty GeometryCollection Faraway point false false false +Empty GeometryCollection Line going through left and right square false false false +Empty GeometryCollection NULL NULL NULL NULL +Empty GeometryCollection Point middle of Left Square false false false +Empty GeometryCollection Point middle of Right Square false false false +Empty GeometryCollection Square (left) false false false +Empty GeometryCollection Square (right) false false false +Empty GeometryCollection Square overlapping left and right square false false false +Empty LineString Empty GeometryCollection false false false +Empty LineString Empty LineString false false false +Empty LineString Faraway point false false false +Empty LineString Line going through left and right square false false false +Empty LineString NULL NULL NULL NULL +Empty LineString Point middle of Left Square false false false +Empty LineString Point middle of Right Square false false false +Empty LineString Square (left) false false false +Empty LineString Square (right) false false false +Empty LineString Square overlapping left and right square false false false +Faraway point Empty GeometryCollection false false false +Faraway point Empty LineString false false false Faraway point Faraway point true true true Faraway point Line going through left and right square false false false Faraway point NULL NULL NULL NULL @@ -867,6 +1078,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) false false false Faraway point Square (right) false false false Faraway point Square overlapping left and right square false false false +Line going through left and right square Empty GeometryCollection false false false +Line going through left and right square Empty LineString false false false Line going through left and right square Faraway point false false false Line going through left and right square Line going through left and right square true true true Line going through left and right square NULL NULL NULL NULL @@ -875,6 +1088,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) false false true Line going through left and right square Square (right) false false true Line going through left and right square Square overlapping left and right square false false true +NULL Empty GeometryCollection NULL NULL NULL +NULL Empty LineString NULL NULL NULL NULL Faraway point NULL NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL NULL NULL @@ -883,6 +1098,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL NULL Square (right) NULL NULL NULL NULL Square overlapping left and right square NULL NULL NULL +Point middle of Left Square Empty GeometryCollection false false false +Point middle of Left Square Empty LineString false false false Point middle of Left Square Faraway point false false false Point middle of Left Square Line going through left and right square false true true Point middle of Left Square NULL NULL NULL NULL @@ -891,6 +1108,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) false true true Point middle of Left Square Square (right) false false false Point middle of Left Square Square overlapping left and right square false false false +Point middle of Right Square Empty GeometryCollection false false false +Point middle of Right Square Empty LineString false false false Point middle of Right Square Faraway point false false false Point middle of Right Square Line going through left and right square false true true Point middle of Right Square NULL NULL NULL NULL @@ -899,6 +1118,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) false false false Point middle of Right Square Square (right) false true true Point middle of Right Square Square overlapping left and right square false true true +Square (left) Empty GeometryCollection false false false +Square (left) Empty LineString false false false Square (left) Faraway point false false false Square (left) Line going through left and right square false false true Square (left) NULL NULL NULL NULL @@ -907,6 +1128,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) true true true Square (left) Square (right) false false true Square (left) Square overlapping left and right square false false true +Square (right) Empty GeometryCollection false false false +Square (right) Empty LineString false false false Square (right) Faraway point false false false Square (right) Line going through left and right square false false true Square (right) NULL NULL NULL NULL @@ -915,6 +1138,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) false false true Square (right) Square (right) true true true Square (right) Square overlapping left and right square false false true +Square overlapping left and right square Empty GeometryCollection false false false +Square overlapping left and right square Empty LineString false false false Square overlapping left and right square Faraway point false false false Square overlapping left and right square Line going through left and right square false false true Square overlapping left and right square NULL NULL NULL NULL @@ -936,6 +1161,28 @@ FROM geog_operators_test a JOIN geog_operators_test b ON (1=1) ORDER BY a.dsc, b.dsc ---- +Empty GeometryCollection Empty GeometryCollection false false false +Empty GeometryCollection Empty LineString false false false +Empty GeometryCollection Faraway point false false false +Empty GeometryCollection Line going through left and right square false false false +Empty GeometryCollection NULL NULL NULL NULL +Empty GeometryCollection Point middle of Left Square false false false +Empty GeometryCollection Point middle of Right Square false false false +Empty GeometryCollection Square (left) false false false +Empty GeometryCollection Square (right) false false false +Empty GeometryCollection Square overlapping left and right square false false false +Empty LineString Empty GeometryCollection false false false +Empty LineString Empty LineString false false false +Empty LineString Faraway point false false false +Empty LineString Line going through left and right square false false false +Empty LineString NULL NULL NULL NULL +Empty LineString Point middle of Left Square false false false +Empty LineString Point middle of Right Square false false false +Empty LineString Square (left) false false false +Empty LineString Square (right) false false false +Empty LineString Square overlapping left and right square false false false +Faraway point Empty GeometryCollection false false false +Faraway point Empty LineString false false false Faraway point Faraway point true true true Faraway point Line going through left and right square false false false Faraway point NULL NULL NULL NULL @@ -944,6 +1191,8 @@ Faraway point Point middle of Right Square Faraway point Square (left) false false false Faraway point Square (right) false false false Faraway point Square overlapping left and right square false false false +Line going through left and right square Empty GeometryCollection false false false +Line going through left and right square Empty LineString false false false Line going through left and right square Faraway point false false false Line going through left and right square Line going through left and right square true true true Line going through left and right square NULL NULL NULL NULL @@ -952,6 +1201,8 @@ Line going through left and right square Point middle of Right Square Line going through left and right square Square (left) true true true Line going through left and right square Square (right) true true true Line going through left and right square Square overlapping left and right square true true true +NULL Empty GeometryCollection NULL NULL NULL +NULL Empty LineString NULL NULL NULL NULL Faraway point NULL NULL NULL NULL Line going through left and right square NULL NULL NULL NULL NULL NULL NULL NULL @@ -960,6 +1211,8 @@ NULL Point middle of Right Square NULL Square (left) NULL NULL NULL NULL Square (right) NULL NULL NULL NULL Square overlapping left and right square NULL NULL NULL +Point middle of Left Square Empty GeometryCollection false false false +Point middle of Left Square Empty LineString false false false Point middle of Left Square Faraway point false false false Point middle of Left Square Line going through left and right square true true true Point middle of Left Square NULL NULL NULL NULL @@ -968,6 +1221,8 @@ Point middle of Left Square Point middle of Right Square Point middle of Left Square Square (left) true true true Point middle of Left Square Square (right) true true true Point middle of Left Square Square overlapping left and right square true true true +Point middle of Right Square Empty GeometryCollection false false false +Point middle of Right Square Empty LineString false false false Point middle of Right Square Faraway point false false false Point middle of Right Square Line going through left and right square true true true Point middle of Right Square NULL NULL NULL NULL @@ -976,6 +1231,8 @@ Point middle of Right Square Point middle of Right Square Point middle of Right Square Square (left) true true true Point middle of Right Square Square (right) true true true Point middle of Right Square Square overlapping left and right square true true true +Square (left) Empty GeometryCollection false false false +Square (left) Empty LineString false false false Square (left) Faraway point false false false Square (left) Line going through left and right square true true true Square (left) NULL NULL NULL NULL @@ -984,6 +1241,8 @@ Square (left) Point middle of Right Square Square (left) Square (left) true true true Square (left) Square (right) true true true Square (left) Square overlapping left and right square true true true +Square (right) Empty GeometryCollection false false false +Square (right) Empty LineString false false false Square (right) Faraway point false false false Square (right) Line going through left and right square true true true Square (right) NULL NULL NULL NULL @@ -992,6 +1251,8 @@ Square (right) Point middle of Right Square Square (right) Square (left) true true true Square (right) Square (right) true true true Square (right) Square overlapping left and right square true true true +Square overlapping left and right square Empty GeometryCollection false false false +Square overlapping left and right square Empty LineString false false false Square overlapping left and right square Faraway point false false false Square overlapping left and right square Line going through left and right square true true true Square overlapping left and right square NULL NULL NULL NULL @@ -1010,6 +1271,8 @@ SELECT FROM geog_operators_test ORDER BY dsc ---- +Empty GeometryCollection GEOMETRYCOLLECTION EMPTY GEOMETRYCOLLECTION EMPTY +Empty LineString LINESTRING EMPTY LINESTRING EMPTY Faraway point POINT (5 5) POINT (5 5) Line going through left and right square LINESTRING (-0.5 0.5, -0.00000000000000009939611878359099 0.5000190382262164, 0.5 0.5) LINESTRING (-0.5 0.5, -0.25000000036247944 0.500014278647005, -0.00000000000000009939611878359099 0.5000190382262164, 0.2500000003624792 0.5000142786470051, 0.5 0.5) NULL NULL NULL diff --git a/pkg/sql/sem/builtins/geo_builtins.go b/pkg/sql/sem/builtins/geo_builtins.go index dc932f183174..6139d95774c8 100644 --- a/pkg/sql/sem/builtins/geo_builtins.go +++ b/pkg/sql/sem/builtins/geo_builtins.go @@ -851,7 +851,17 @@ var geoBuiltins = map[string]builtinDefinition{ if err != nil { return nil, err } - return tree.NewDInt(tree.DInt(len(t.FlatCoords()) / t.Stride())), nil + switch t := t.(type) { + case *geom.GeometryCollection: + // FlatCoords() does not work on GeometryCollection. + numPoints := 0 + for _, g := range t.Geoms() { + numPoints += len(g.FlatCoords()) / g.Stride() + } + return tree.NewDInt(tree.DInt(numPoints)), nil + default: + return tree.NewDInt(tree.DInt(len(t.FlatCoords()) / t.Stride())), nil + } }, types.Int, infoBuilder{ @@ -1241,6 +1251,9 @@ Note ST_Perimeter is only valid for Polygon - use ST_Length for LineString.`, func(ctx *tree.EvalContext, a, b *tree.DGeometry) (tree.Datum, error) { ret, err := geomfn.MinDistance(a.Geometry, b.Geometry) if err != nil { + if geo.IsEmptyGeometryError(err) { + return tree.DNull, nil + } return nil, err } return tree.NewDFloat(tree.DFloat(ret)), nil @@ -1255,6 +1268,9 @@ Note ST_Perimeter is only valid for Polygon - use ST_Length for LineString.`, func(ctx *tree.EvalContext, a *tree.DGeography, b *tree.DGeography) (tree.Datum, error) { ret, err := geogfn.Distance(a.Geography, b.Geography, geogfn.UseSpheroid) if err != nil { + if geo.IsEmptyGeometryError(err) { + return tree.DNull, nil + } return nil, err } return tree.NewDFloat(tree.DFloat(ret)), nil @@ -1281,6 +1297,9 @@ Note ST_Perimeter is only valid for Polygon - use ST_Length for LineString.`, ret, err := geogfn.Distance(a.Geography, b.Geography, toUseSphereOrSpheroid(useSpheroid)) if err != nil { + if geo.IsEmptyGeometryError(err) { + return tree.DNull, nil + } return nil, err } return tree.NewDFloat(tree.DFloat(ret)), nil @@ -1299,6 +1318,9 @@ Note ST_Perimeter is only valid for Polygon - use ST_Length for LineString.`, func(ctx *tree.EvalContext, a, b *tree.DGeometry) (tree.Datum, error) { ret, err := geomfn.MaxDistance(a.Geometry, b.Geometry) if err != nil { + if geo.IsEmptyGeometryError(err) { + return tree.DNull, nil + } return nil, err } return tree.NewDFloat(tree.DFloat(ret)), nil