Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
99411: roachtest: make `follower-reads` test use protobuf encoded requests r=aadityasondhi a=aadityasondhi

Previously, the roachtest for
`follower-reads/mixed-version/single-region` used JSON encoding for the `ts/query` endpoint. This caused the client to send the newly updated `timerseries` proto to an older cluster in JSON format. Although this was defined as an `optional` field in the proto, since it was encoded in JSON, that information was lost over the wire.

In other places, such as DBConsole, we expect the client to be sending protobuf encoded messages and not JSON. In the same way, roachtests (another client) should do the same. When this message is sent as a protobuf, the `optional` tag of the field is encoded in the message and the server is able to process it as such.

There are related links in the issue:
#99117.

Fixes #99117.

Release note: None

99583: tree: apply functions to TEXT expressions compared with the @@ operator r=DrewKimball a=msirek

The TSQuery and TSVector "matches" operator "`@@"` returns different results on CRDB vs. Postgres when one of the arguments is a TEXT expression. CRDB always applies a CAST, and only for constants. The rules at https://www.postgresql.org/docs/current/textsearch-intro.html#TEXTSEARCH-MATCHING specify:
> The form text `@@` tsquery is equivalent to to_tsvector(x) `@@` y.
> The form text `@@` text is equivalent to to_tsvector(x) `@@` plainto_tsquery(y).

This PR adds these implicit function calls in these "matches" comparison expressions during type checking as well as a cast of TEXT to TSQuery when the other argument is a TSVector, which allows variable expressions to be handled.

Fixes #98875
Fixes #98804

Release note (bug fix): This allows the text search `@@` ("matches") operator to work with variable expressions and fixes incorrect results when one of the arguments is a TEXT expression and the other argument is a TEXT or TSQuery expression.

99721: roachtest: fix distsql version issue with mt upgrade roachtest r=ajstorm a=healthy-pod

This code change only fixes one issue with the test by not validating the cluster in the intermediate state because the SQL servers will have different DistSQL versions.

Release note: None
Epic: none

Co-authored-by: Aaditya Sondhi <[email protected]>
Co-authored-by: Mark Sirek <[email protected]>
Co-authored-by: healthy-pod <[email protected]>
  • Loading branch information
4 people committed Mar 28, 2023
4 parents 3f113fc + b0659ec + 60ad5fb + 89a4ecb commit c1b1b4c
Show file tree
Hide file tree
Showing 7 changed files with 195 additions and 15 deletions.
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/follower_reads.go
Original file line number Diff line number Diff line change
Expand Up @@ -730,7 +730,7 @@ func verifyHighFollowerReadRatios(
}

var response tspb.TimeSeriesQueryResponse
if err := httputil.PostJSON(http.Client{}, url, &request, &response); err != nil {
if err := httputil.PostProtobuf(ctx, http.Client{}, url, &request, &response); err != nil {
t.Fatal(err)
}

Expand Down
13 changes: 9 additions & 4 deletions pkg/cmd/roachtest/tests/multitenant_upgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -226,6 +226,15 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
`pq: error validating the version of one or more SQL server instances: validate cluster version failed: some tenant pods running on binary less than 23.1`,
"SET CLUSTER SETTING version = crdb_internal.node_executable_version()")

// Note that here we'd like to validate that the first tenant 11 server can
// query the storage cluster. The problem however, is that due to #88927,
// they can't because they're at different DistSQL versions. We plan to never change
// the DistSQL version again so once we have 23.1 images to test against we should
// add a check in here that we're able to query from tenant 11 first server.
t.Status("stop the second tenant 11 server and restart it on the new binary")
tenant11b.stop(ctx, t, c)
tenant11b.start(ctx, t, c, currentBinary)

t.Status("verify that the first tenant 11 server can now query the storage cluster")
{
verifySQL(t, tenant11a.pgURL,
Expand All @@ -235,10 +244,6 @@ func runMultiTenantUpgrade(ctx context.Context, t test.Test, c cluster.Cluster,
withResults([][]string{{initialVersion}}))
}

t.Status("stop the second tenant 11 server and restart it on the new binary")
tenant11b.stop(ctx, t, c)
tenant11b.start(ctx, t, c, currentBinary)

t.Status("verify the second tenant 11 server works with the new binary")
{
verifySQL(t, tenant11b.pgURL,
Expand Down
64 changes: 64 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/tsvector
Original file line number Diff line number Diff line change
Expand Up @@ -360,3 +360,67 @@ LIMIT
2
----
0

subtest 98804_regression_test

statement ok
RESET default_text_search_config

statement ok
CREATE TABLE ab (a TEXT, b TEXT)

statement ok
INSERT INTO ab VALUES('fat rats', 'fat cats chased fat, out of shape rats');

query B
SELECT a @@ b FROM ab
----
false

query B
SELECT b @@ a FROM ab
----
true

query B
SELECT 'fat rats' @@ b FROM ab
----
false

query B
SELECT b @@ 'fat rats' FROM ab
----
true

query B
SELECT a @@ 'fat cats ate fat bats' FROM ab
----
false

query B
SELECT 'fat cats ate fat bats' @@ a FROM ab
----
false

statement error pq: syntax error in TSQuery: fat cats chased fat, out of shape rats
SELECT b @@ a::tsvector FROM ab

statement error pq: syntax error in TSQuery: fat cats chased fat, out of shape rats
SELECT a::tsvector @@ b FROM ab

query B
SELECT 'fat bat cat' @@ 'bats fats'
----
true

query B
SELECT 'bats fats' @@ 'fat bat cat'
----
false

statement error pq: syntax error in TSQuery: fat cats chased fat, out of shape rats
SELECT 'fat cats chased fat, out of shape rats' @@ 'fat rats'::tsvector

statement error pq: syntax error in TSQuery: fat cats chased fat, out of shape rats
SELECT 'fat rats'::tsvector @@ 'fat cats chased fat, out of shape rats'

2 changes: 1 addition & 1 deletion pkg/sql/sem/tree/overload.go
Original file line number Diff line number Diff line change
Expand Up @@ -997,7 +997,7 @@ func (s *overloadTypeChecker) typeCheckOverloadedExprs(
return err
}

// The fourth heuristic is to prefer candidates that accepts the "best"
// The fourth heuristic is to prefer candidates that accept the "best"
// mutual type in the resolvable type set of all constants.
if bestConstType, ok := commonConstantType(s.exprs, s.constIdxs); ok {
// In case all overloads are filtered out at this step,
Expand Down
85 changes: 76 additions & 9 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -2204,6 +2204,38 @@ func typeCheckComparisonOp(
_, leftIsTuple := foldedLeft.(*Tuple)
_, rightIsTuple := foldedRight.(*Tuple)
_, rightIsSubquery := foldedRight.(SubqueryExpr)
var tsMatchesWithText bool
var typedLeft, typedRight TypedExpr
var leftFamily, rightFamily types.Family
var err error

// Do an initial check for TEXT @@ XXX special cases which might need to
// inject a to_tsvector or plainto_tsquery function call.
if op.Symbol == treecmp.TSMatches {
if switched {
// The order of operators matters as to which function call to apply.
foldedLeft, foldedRight = foldedRight, foldedLeft
switched = false
}
disallowSwitch = true
typedLeft, err = foldedLeft.TypeCheck(ctx, semaCtx, types.Any)
if err != nil {
sigWithErr := fmt.Sprintf(compExprsFmt, left, op, right, err)
return nil, nil, nil, false,
pgerror.Newf(pgcode.InvalidParameterValue, unsupportedCompErrFmt, sigWithErr)
}
typedRight, err = foldedRight.TypeCheck(ctx, semaCtx, types.Any)
if err != nil {
sigWithErr := fmt.Sprintf(compExprsFmt, left, op, right, err)
return nil, nil, nil, false,
pgerror.Newf(pgcode.InvalidParameterValue, unsupportedCompErrFmt, sigWithErr)
}
leftFamily = typedLeft.ResolvedType().Family()
rightFamily = typedRight.ResolvedType().Family()
if leftFamily == types.StringFamily || rightFamily == types.StringFamily {
tsMatchesWithText = true
}
}

handleTupleTypeMismatch := false
switch {
Expand All @@ -2227,7 +2259,7 @@ func typeCheckComparisonOp(
pgerror.Newf(pgcode.InvalidParameterValue, unsupportedCompErrFmt, sig)
}

typedLeft := typedSubExprs[0]
typedLeft = typedSubExprs[0]
typedSubExprs = typedSubExprs[1:]

rightTuple.typ = types.MakeTuple(make([]*types.T, len(typedSubExprs)))
Expand All @@ -2241,7 +2273,7 @@ func typeCheckComparisonOp(
return typedLeft, rightTuple, fn, false, nil

case foldedOp.Symbol == treecmp.In && rightIsSubquery:
typedLeft, err := foldedLeft.TypeCheck(ctx, semaCtx, types.Any)
typedLeft, err = foldedLeft.TypeCheck(ctx, semaCtx, types.Any)
if err != nil {
sigWithErr := fmt.Sprintf(compExprsFmt, left, op, right, err)
return nil, nil, nil, false,
Expand All @@ -2257,14 +2289,14 @@ func typeCheckComparisonOp(
}

desired := types.MakeTuple([]*types.T{typ})
typedRight, err := foldedRight.TypeCheck(ctx, semaCtx, desired)
typedRight, err = foldedRight.TypeCheck(ctx, semaCtx, desired)
if err != nil {
sigWithErr := fmt.Sprintf(compExprsFmt, left, op, right, err)
return nil, nil, nil, false,
pgerror.Newf(pgcode.InvalidParameterValue, unsupportedCompErrFmt, sigWithErr)
}

if err := typeCheckSubqueryWithIn(
if err = typeCheckSubqueryWithIn(
typedLeft.ResolvedType(), typedRight.ResolvedType(),
); err != nil {
return nil, nil, nil, false, err
Expand All @@ -2279,23 +2311,58 @@ func typeCheckComparisonOp(
pgerror.Newf(pgcode.InvalidParameterValue, unsupportedCompErrFmt, sig)
}
// Using non-folded left and right to avoid having to swap later.
typedLeft, typedRight, err := typeCheckTupleComparison(ctx, semaCtx, op, left.(*Tuple), right.(*Tuple))
typedLeft, typedRight, err = typeCheckTupleComparison(ctx, semaCtx, op, left.(*Tuple), right.(*Tuple))
if err != nil {
return nil, nil, nil, false, err
}
return typedLeft, typedRight, fn, false, nil

case leftIsTuple || rightIsTuple:
var errLeft, errRight error
// Tuple must compare with a tuple type, as handled above.
typedLeft, errLeft := foldedLeft.TypeCheck(ctx, semaCtx, types.Any)
typedRight, errRight := foldedRight.TypeCheck(ctx, semaCtx, types.Any)
typedLeft, errLeft = foldedLeft.TypeCheck(ctx, semaCtx, types.Any)
typedRight, errRight = foldedRight.TypeCheck(ctx, semaCtx, types.Any)
if errLeft == nil && errRight == nil &&
((typedLeft.ResolvedType().Family() == types.TupleFamily &&
typedRight.ResolvedType().Family() != types.TupleFamily) ||
(typedRight.ResolvedType().Family() == types.TupleFamily &&
typedLeft.ResolvedType().Family() != types.TupleFamily)) {
handleTupleTypeMismatch = true
}
case tsMatchesWithText:
// Apply rules from:
// https://www.postgresql.org/docs/current/textsearch-intro.html#TEXTSEARCH-MATCHING
// Perform the following type conversions:
// initial | result
// -------------------------------------------------------------------------
// a::TEXT @@ b::TEXT | to_tsvector(a) @@ plainto_tsquery(b)
// a::TEXT @@ b::TSQUERY | to_tsvector(a) @@ b
// a::TSQUERY @@ b::TEXT | a @@ to_tsvector(b)
// a::TSVECTOR @@ b::TEXT | a @@ b::TSQUERY
// a::TEXT @@ b::TSVECTOR | a::TSQUERY @@ b
if leftFamily == types.StringFamily {
if rightFamily == types.StringFamily || rightFamily == types.TSQueryFamily {
leftExprs := make(Exprs, 1)
leftExprs[0] = typedLeft
foldedLeft = &FuncExpr{Func: WrapFunction("to_tsvector"), Exprs: leftExprs, AggType: GeneralAgg}
} else if rightFamily == types.TSVectorFamily {
foldedLeft = &CastExpr{Expr: typedLeft, Type: types.TSQuery, SyntaxMode: CastShort}
}
}

funcName := "plainto_tsquery"
if rightFamily == types.StringFamily {
if leftFamily == types.StringFamily || leftFamily == types.TSQueryFamily {
if leftFamily == types.TSQueryFamily {
funcName = "to_tsvector"
}
rightExprs := make(Exprs, 1)
rightExprs[0] = typedRight
foldedRight = &FuncExpr{Func: WrapFunction(funcName), Exprs: rightExprs, AggType: GeneralAgg}
} else if leftFamily == types.TSVectorFamily {
foldedRight = &CastExpr{Expr: typedRight, Type: types.TSQuery, SyntaxMode: CastShort}
}
}
}

// For comparisons, we do not stimulate the typing of untyped NULL with the
Expand Down Expand Up @@ -2356,8 +2423,8 @@ func typeCheckComparisonOp(
}
leftReturn := leftExpr.ResolvedType()
rightReturn := rightExpr.ResolvedType()
leftFamily := leftReturn.Family()
rightFamily := rightReturn.Family()
leftFamily = leftReturn.Family()
rightFamily = rightReturn.Family()

// Return early if at least one overload is possible, NULL is an argument,
// and none of the overloads accept NULL.
Expand Down
7 changes: 7 additions & 0 deletions pkg/sql/sem/tree/type_check_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,6 +213,13 @@ func TestTypeCheck(t *testing.T) {

// String preference.
{`st_geomfromgeojson($1)`, `st_geomfromgeojson($1:::STRING):::GEOMETRY`},

// TSQuery and TSVector
{`'a' @@ 'b'`, `to_tsvector('a':::STRING) @@ plainto_tsquery('b':::STRING)`},
{`'a' @@ 'b':::TSQUERY`, `to_tsvector('a':::STRING) @@ '''b''':::TSQUERY`},
{`'a':::TSQUERY @@ 'b'`, `'''a''':::TSQUERY @@ to_tsvector('b':::STRING)`},
{`'a' @@ 'b':::TSVECTOR`, `'a':::STRING::TSQUERY @@ '''b''':::TSVECTOR`},
{`'a':::TSVECTOR @@ 'b'`, `'''a''':::TSVECTOR @@ 'b':::STRING::TSQUERY`},
}
ctx := context.Background()
for _, d := range testData {
Expand Down
37 changes: 37 additions & 0 deletions pkg/util/httputil/http.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package httputil

import (
"bytes"
"context"
"io"
"net/http"
"strconv"
Expand Down Expand Up @@ -110,6 +111,42 @@ func PostJSONWithRequest(
return doJSONRequest(httpClient, req, response)
}

// PostProtobuf uses the supplied client to POST request to the URL specified by
// the parameters and unmarshal the result into response, using a
// protobuf-encoded request body.
func PostProtobuf(
ctx context.Context, httpClient http.Client, path string, request, response protoutil.Message,
) error {
buf, err := protoutil.Marshal(request)
if err != nil {
return err
}
reader := bytes.NewReader(buf)
req, err := http.NewRequestWithContext(ctx, "POST", path, reader)
if err != nil {
return err
}
if timeout := httpClient.Timeout; timeout > 0 {
req.Header.Set("Grpc-Timeout", strconv.FormatInt(timeout.Nanoseconds(), 10)+"n")
}
req.Header.Set(AcceptHeader, ProtoContentType)
req.Header.Set(ContentTypeHeader, ProtoContentType)
resp, err := httpClient.Do(req)
if err != nil {
return err
}
defer resp.Body.Close()
b, err := io.ReadAll(resp.Body)
if contentType := resp.Header.Get(ContentTypeHeader); !(resp.StatusCode == http.StatusOK && contentType == ProtoContentType) {
// NB: errors.Wrapf(nil, ...) returns nil.
// nolint:errwrap
return errors.Errorf(
"status: %s, content-type: %s, body: %s, error: %v", resp.Status, contentType, b, err,
)
}
return protoutil.Unmarshal(b, response)
}

func doJSONRequest(
httpClient http.Client, req *http.Request, response protoutil.Message,
) (*http.Response, error) {
Expand Down

0 comments on commit c1b1b4c

Please sign in to comment.