Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
97685: sql: add default_text_search_config r=jordanlewis a=jordanlewis

Updates: #41288
Epic: CRDB-22357

All but the last commit are from #92966 and #97677.


    This commit adds the default_text_search_config variable for the tsearch
    package, which allows the user to set a default configuration for the
    text search builtin functions that take configurations, such as
    to_tsvector and to_tsquery. The default for this configuration variable
    is 'english', as it is in Postgres.

    Release note (sql change): add the default_text_search_config variable
    for compatibility with the single-argument variants of the text search
    functions to_tsvector, to_tsquery, phraseto_tsquery, and
    plainto_tsquery, which use the value of default_text_search_config
    instead of expecting one to be included as in the two-argument variants.
    The default value of this setting is 'english'.

99045: roachtest: set 30m timeout for all disk stall roachtests r=nicktrav a=jbowens

This commit sets a new 30m timeout for all disk stall roachtests. Previously,
the FUSE filesystem variants had no timeout and inherited the default 10h
timeout. The other variants had a 20m timeout, which has been observed to be
too short due to upreplication latency.

Informs #98904.
Informs #98886.
Epic: None
Release note: None


99057: sql: check replace view columns earlier r=rharding6373 a=rharding6373

Before this change, we could encounter internal errors while attempting to add result columns during a `CREATE OR REPLACE VIEW` if the number of columns in the new view was less than the number of columns in the old view. This led to an inconsistency with postgres, which would only return the error `cannot drop columns from view`.

This PR moves the check comparing the number of columns before and after the view replacement earlier so that the correct error returns.

Co-authored-by: [email protected]

Fixes: #99000
Epic: None

Release note (bug fix): Fixes an internal error that can occur when `CREATE OR REPLACE VIEW` replaces a view with fewer columns and another entity depended on the view.

Co-authored-by: Jordan Lewis <[email protected]>
Co-authored-by: Jackson Owens <[email protected]>
Co-authored-by: craig[bot] <[email protected]>
  • Loading branch information
3 people committed Mar 21, 2023
4 parents 77029b1 + 8ca25df + 5f4fe6e + b80feac commit b26b338
Show file tree
Hide file tree
Showing 17 changed files with 202 additions and 10 deletions.
16 changes: 12 additions & 4 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -888,14 +888,22 @@ available replica will error.</p>
<table>
<thead><tr><th>Function &rarr; Returns</th><th>Description</th><th>Volatility</th></tr></thead>
<tbody>
<tr><td><a name="phraseto_tsquery"></a><code>phraseto_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the specified or default configuration. The &lt;-&gt; operator is inserted between each token in the input.</p>
<tr><td><a name="phraseto_tsquery"></a><code>phraseto_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the specified configuration. The &lt;-&gt; operator is inserted between each token in the input.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="plainto_tsquery"></a><code>plainto_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the specified or default configuration. The &amp; operator is inserted between each token in the input.</p>
<tr><td><a name="phraseto_tsquery"></a><code>phraseto_tsquery(text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the default configuration. The &lt;-&gt; operator is inserted between each token in the input.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="plainto_tsquery"></a><code>plainto_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the specified configuration. The &amp; operator is inserted between each token in the input.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="to_tsquery"></a><code>to_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts the input text into a tsquery by normalizing each word in the input according to the specified or default configuration. The input must already be formatted like a tsquery, in other words, subsequent tokens must be connected by a tsquery operator (&amp;, |, &lt;-&gt;, !).</p>
<tr><td><a name="plainto_tsquery"></a><code>plainto_tsquery(text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts text to a tsquery, normalizing words according to the default configuration. The &amp; operator is inserted between each token in the input.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="to_tsquery"></a><code>to_tsquery(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts the input text into a tsquery by normalizing each word in the input according to the specified configuration. The input must already be formatted like a tsquery, in other words, subsequent tokens must be connected by a tsquery operator (&amp;, |, &lt;-&gt;, !).</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="to_tsvector"></a><code>to_tsvector(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsvector</code></td><td><span class="funcdesc"><p>Converts text to a tsvector, normalizing words according to the specified or default configuration. Position information is included in the result.</p>
<tr><td><a name="to_tsquery"></a><code>to_tsquery(text: <a href="string.html">string</a>) &rarr; tsquery</code></td><td><span class="funcdesc"><p>Converts the input text into a tsquery by normalizing each word in the input according to the default configuration. The input must already be formatted like a tsquery, in other words, subsequent tokens must be connected by a tsquery operator (&amp;, |, &lt;-&gt;, !).</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="to_tsvector"></a><code>to_tsvector(config: <a href="string.html">string</a>, text: <a href="string.html">string</a>) &rarr; tsvector</code></td><td><span class="funcdesc"><p>Converts text to a tsvector, normalizing words according to the specified configuration. Position information is included in the result.</p>
</span></td><td>Immutable</td></tr>
<tr><td><a name="to_tsvector"></a><code>to_tsvector(text: <a href="string.html">string</a>) &rarr; tsvector</code></td><td><span class="funcdesc"><p>Converts text to a tsvector, normalizing words according to the default configuration. Position information is included in the result.</p>
</span></td><td>Stable</td></tr>
<tr><td><a name="ts_parse"></a><code>ts_parse(parser_name: <a href="string.html">string</a>, document: <a href="string.html">string</a>) &rarr; tuple{int AS tokid, string AS token}</code></td><td><span class="funcdesc"><p>ts_parse parses the given document and returns a series of records, one for each token produced by parsing. Each record includes a tokid showing the assigned token type and a token which is the text of the token.</p>
</span></td><td>Stable</td></tr></tbody>
</table>
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/disk_stall.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func registerDiskStalledDetection(r registry.Registry) {
Name: fmt.Sprintf("disk-stalled/%s", name),
Owner: registry.OwnerStorage,
Cluster: r.MakeClusterSpec(4, spec.ReuseNone()),
Timeout: 20 * time.Minute,
Timeout: 30 * time.Minute,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runDiskStalledDetection(ctx, t, c, makeStaller(t, c), true /* doStall */)
},
Expand All @@ -77,6 +77,7 @@ func registerDiskStalledDetection(r registry.Registry) {
),
Owner: registry.OwnerStorage,
Cluster: r.MakeClusterSpec(4, spec.ReuseNone()),
Timeout: 30 * time.Minute,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
runDiskStalledDetection(ctx, t, c, &fuseDiskStaller{
t: t,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -546,6 +546,7 @@ go_library(
"//pkg/util/tracing",
"//pkg/util/tracing/collector",
"//pkg/util/tracing/tracingpb",
"//pkg/util/tsearch",
"//pkg/util/uint128",
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v3//:apd",
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/create_view.go
Original file line number Diff line number Diff line change
Expand Up @@ -596,6 +596,11 @@ func (p *planner) replaceViewDesc(
toReplace.ViewQuery = updatedQuery
}

// Check that the new view has at least as many columns as the old view before
// adding result columns.
if len(n.columns) < len(toReplace.ClusterVersion().Columns) {
return nil, pgerror.Newf(pgcode.InvalidTableDefinition, "cannot drop columns from view")
}
// Reset the columns to add the new result columns onto.
toReplace.Columns = make([]descpb.ColumnDescriptor, 0, len(n.columns))
toReplace.NextColumnID = 0
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3490,6 +3490,10 @@ func (m *sessionDataMutator) SetAllowRoleMembershipsToChangeDuringTransaction(va
m.data.AllowRoleMembershipsToChangeDuringTransaction = val
}

func (m *sessionDataMutator) SetDefaultTextSearchConfig(val string) {
m.data.DefaultTextSearchConfig = val
}

// Utility functions related to scrubbing sensitive information on SQL Stats.

// quantizeCounts ensures that the Count field in the
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5075,6 +5075,7 @@ declare_cursor_statement_timeout_enabled on
default_int_size 8
default_table_access_method heap
default_tablespace ·
default_text_search_config pg_catalog.english
default_transaction_isolation serializable
default_transaction_priority normal
default_transaction_quality_of_service regular
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2632,6 +2632,7 @@ declare_cursor_statement_timeout_enabled on NULL
default_int_size 8 NULL NULL NULL string
default_table_access_method heap NULL NULL NULL string
default_tablespace · NULL NULL NULL string
default_text_search_config pg_catalog.english NULL NULL NULL string
default_transaction_isolation serializable NULL NULL NULL string
default_transaction_priority normal NULL NULL NULL string
default_transaction_quality_of_service regular NULL NULL NULL string
Expand Down Expand Up @@ -2782,6 +2783,7 @@ declare_cursor_statement_timeout_enabled on NULL
default_int_size 8 NULL user NULL 8 8
default_table_access_method heap NULL user NULL heap heap
default_tablespace · NULL user NULL · ·
default_text_search_config pg_catalog.english NULL user NULL pg_catalog.english pg_catalog.english
default_transaction_isolation serializable NULL user NULL default default
default_transaction_priority normal NULL user NULL normal normal
default_transaction_quality_of_service regular NULL user NULL regular regular
Expand Down Expand Up @@ -2928,6 +2930,7 @@ declare_cursor_statement_timeout_enabled NULL NULL NULL
default_int_size NULL NULL NULL NULL NULL
default_table_access_method NULL NULL NULL NULL NULL
default_tablespace NULL NULL NULL NULL NULL
default_text_search_config NULL NULL NULL NULL NULL
default_transaction_isolation NULL NULL NULL NULL NULL
default_transaction_priority NULL NULL NULL NULL NULL
default_transaction_quality_of_service NULL NULL NULL NULL NULL
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ declare_cursor_statement_timeout_enabled on
default_int_size 8
default_table_access_method heap
default_tablespace ·
default_text_search_config pg_catalog.english
default_transaction_isolation serializable
default_transaction_priority normal
default_transaction_quality_of_service regular
Expand Down
23 changes: 23 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/tsvector
Original file line number Diff line number Diff line change
Expand Up @@ -275,3 +275,26 @@ SELECT * FROM to_tsquery('simple', 'a | b & c <-> d')

query error syntax
SELECT * FROM to_tsquery('simple', 'Hello this is a parsi-ng t.est 1.234 4 case324')

# Test default variants of the to_ts* functions.

query T
SHOW default_text_search_config
----
pg_catalog.english

query T
SELECT to_tsvector('Hello I am a potato')
----
'hello':1 'potato':5

statement error text search configuration \"blah\" does not exist
SET default_text_search_config = 'blah'

statement ok
SET default_text_search_config = 'spanish'

query T
SELECT to_tsvector('Hello I am a potato')
----
'am':3 'hell':1 'i':2 'potat':5
22 changes: 22 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/views
Original file line number Diff line number Diff line change
Expand Up @@ -1813,3 +1813,25 @@ SELECT * FROM v;

statement ok
SET DATABASE = test;

# When replacing a view with fewer columns, an error about dropping columns from
# views preempts any errors about dependencies.
subtest regression_99000

statement ok
CREATE TABLE films (id int PRIMARY KEY, title text, kind text, classification CHAR(1));

statement ok
CREATE VIEW comedies AS
SELECT *
FROM films
WHERE kind = 'Comedy';

statement ok
CREATE VIEW pg_comedies AS
SELECT *
FROM comedies
WHERE classification = 'PG';

statement error pq: cannot drop columns from view
CREATE OR REPLACE VIEW comedies AS SELECT ARRAY[films.*]::string FROM films;
4 changes: 4 additions & 0 deletions pkg/sql/sem/builtins/fixed_oids.go
Original file line number Diff line number Diff line change
Expand Up @@ -2369,6 +2369,10 @@ var builtinOidsArray = []string{
2395: `array_cat_agg(arg1: anyenum[]) -> anyenum[]`,
2396: `array_cat_agg(arg1: tuple[]) -> tuple[]`,
2397: `crdb_internal.update_tenant_resource_limits(tenant_name: string, available_request_units: float, refill_rate: float, max_burst_request_units: float, as_of: timestamp, as_of_consumed_request_units: float) -> int`,
2398: `to_tsquery(text: string) -> tsquery`,
2399: `to_tsvector(text: string) -> tsvector`,
2400: `phraseto_tsquery(text: string) -> tsquery`,
2401: `plainto_tsquery(text: string) -> tsquery`,
}

var builtinOidsBySignature map[string]oid.Oid
Expand Down
73 changes: 69 additions & 4 deletions pkg/sql/sem/builtins/tsearch_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,10 +103,26 @@ var tsearchBuiltins = map[string]builtinDefinition{
}
return &tree.DTSVector{TSVector: vector}, nil
},
Info: "Converts text to a tsvector, normalizing words according to the specified or default configuration. " +
Info: "Converts text to a tsvector, normalizing words according to the specified configuration. " +
"Position information is included in the result.",
Volatility: volatility.Immutable,
},
tree.Overload{
Types: tree.ParamTypes{{Name: "text", Typ: types.String}},
ReturnType: tree.FixedReturnType(types.TSVector),
Fn: func(_ context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
config := tsearch.GetConfigKey(evalCtx.SessionData().DefaultTextSearchConfig)
document := string(tree.MustBeDString(args[0]))
vector, err := tsearch.DocumentToTSVector(config, document)
if err != nil {
return nil, err
}
return &tree.DTSVector{TSVector: vector}, nil
},
Info: "Converts text to a tsvector, normalizing words according to the default configuration. " +
"Position information is included in the result.",
Volatility: volatility.Stable,
},
),
"to_tsquery": makeBuiltin(
tree.FunctionProperties{},
Expand All @@ -123,10 +139,27 @@ var tsearchBuiltins = map[string]builtinDefinition{
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts the input text into a tsquery by normalizing each word in the input according to " +
"the specified or default configuration. The input must already be formatted like a tsquery, in other words, " +
"the specified configuration. The input must already be formatted like a tsquery, in other words, " +
"subsequent tokens must be connected by a tsquery operator (&, |, <->, !).",
Volatility: volatility.Immutable,
},
tree.Overload{
Types: tree.ParamTypes{{Name: "text", Typ: types.String}},
ReturnType: tree.FixedReturnType(types.TSQuery),
Fn: func(_ context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
config := tsearch.GetConfigKey(evalCtx.SessionData().DefaultTextSearchConfig)
input := string(tree.MustBeDString(args[0]))
query, err := tsearch.ToTSQuery(config, input)
if err != nil {
return nil, err
}
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts the input text into a tsquery by normalizing each word in the input according to " +
"the default configuration. The input must already be formatted like a tsquery, in other words, " +
"subsequent tokens must be connected by a tsquery operator (&, |, <->, !).",
Volatility: volatility.Stable,
},
),
"plainto_tsquery": makeBuiltin(
tree.FunctionProperties{},
Expand All @@ -142,10 +175,26 @@ var tsearchBuiltins = map[string]builtinDefinition{
}
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts text to a tsquery, normalizing words according to the specified or default configuration." +
Info: "Converts text to a tsquery, normalizing words according to the specified configuration." +
" The & operator is inserted between each token in the input.",
Volatility: volatility.Immutable,
},
tree.Overload{
Types: tree.ParamTypes{{Name: "text", Typ: types.String}},
ReturnType: tree.FixedReturnType(types.TSQuery),
Fn: func(_ context.Context, evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
config := tsearch.GetConfigKey(evalCtx.SessionData().DefaultTextSearchConfig)
input := string(tree.MustBeDString(args[0]))
query, err := tsearch.PlainToTSQuery(config, input)
if err != nil {
return nil, err
}
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts text to a tsquery, normalizing words according to the default configuration." +
" The & operator is inserted between each token in the input.",
Volatility: volatility.Stable,
},
),
"phraseto_tsquery": makeBuiltin(
tree.FunctionProperties{},
Expand All @@ -161,9 +210,25 @@ var tsearchBuiltins = map[string]builtinDefinition{
}
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts text to a tsquery, normalizing words according to the specified or default configuration." +
Info: "Converts text to a tsquery, normalizing words according to the specified configuration." +
" The <-> operator is inserted between each token in the input.",
Volatility: volatility.Immutable,
},
tree.Overload{
Types: tree.ParamTypes{{Name: "text", Typ: types.String}},
ReturnType: tree.FixedReturnType(types.TSQuery),
Fn: func(evalCtx *eval.Context, args tree.Datums) (tree.Datum, error) {
config := tsearch.GetConfigKey(evalCtx.SessionData().DefaultTextSearchConfig)
input := string(tree.MustBeDString(args[0]))
query, err := tsearch.PhraseToTSQuery(config, input)
if err != nil {
return nil, err
}
return &tree.DTSQuery{TSQuery: query}, nil
},
Info: "Converts text to a tsquery, normalizing words according to the default configuration." +
" The <-> operator is inserted between each token in the input.",
Volatility: volatility.Stable,
},
),
}
4 changes: 4 additions & 0 deletions pkg/sql/sessiondatapb/session_data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,10 @@ message SessionData {
// format should be used for ScanRequests and ReverseScanRequests whenever
// possible.
bool direct_columnar_scans_enabled = 25;
// DefaultTextSearchConfig configures the default text search configuration
// that is used for builtins like to_tsvector and to_tsquery if no
// text search configuration is explicitly passed in.
string default_text_search_config = 26;
}

// DataConversionConfig contains the parameters that influence the output
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/unsupported_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,7 @@ var UnsupportedVars = func(ss ...string) map[string]struct{} {
"debug_print_plan",
"debug_print_rewritten",
"default_statistics_target",
"default_text_search_config",
// "default_text_search_config",
"default_transaction_deferrable",
// "default_transaction_isolation",
// "default_transaction_read_only",
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil/pgdate"
"github.com/cockroachdb/cockroach/pkg/util/tsearch"
"github.com/cockroachdb/errors"
)

Expand Down Expand Up @@ -1263,6 +1264,23 @@ var varGen = map[string]sessionVar{
},
},

// See https://www.postgresql.org/docs/current/runtime-config-client.html#GUC-DEFAULT-TEXT-SEARCH-CONFIG
`default_text_search_config`: {
Set: func(_ context.Context, m sessionDataMutator, s string) error {
if err := tsearch.ValidConfig(s); err != nil {
return err
}
m.SetDefaultTextSearchConfig(s)
return nil
},
Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
return evalCtx.SessionData().DefaultTextSearchConfig, nil
},
GlobalDefault: func(c_ *settings.Values) string {
return "pg_catalog.english"
},
},

// This is read-only in Postgres also.
// See https://www.postgresql.org/docs/14/sql-show.html and
// https://www.postgresql.org/docs/14/locale.html
Expand Down
Loading

0 comments on commit b26b338

Please sign in to comment.