Skip to content

Commit 8004dcd

Browse files
craig[bot]healthy-podDrewKimballchrisseto
committed
104859: pkg/server: initialize tenant version setting in settings watcher r=knz a=healthy-pod This code change removes the early version setting initialization because it is not needed. Initialization now happens in the settings watcher during the initial scan when the tenant logical version is read from the settings table. The setting is only initialized if it's empty / not pre-initialized because some tests pre-initialize it. Release note: None Epic: CRDB-26691 107920: plpgsql: add support for sql exec and INTO statements r=DrewKimball a=DrewKimball #### plpgsql: add parser support for sql exec statements This patch adds support in the parser for directly executing SQL statements in a PLpgSQL routine. Note that postgres disallows this syntax for statements that return rows, but we allow it because execution can handle row-returning statements and it allows running statements like `IMPORT`, which returns rows. SQL statements with an `INTO` clause are also supported; `INTO` allows for the result of a SQL statement to be assigned to one or more PLpgSQL variables. It cannot be used with a statement that does not return rows. Example: ``` CREATE FUNCTION f() RETURNS INT AS $$ DECLARE i INT; BEGIN SET testing_optimizer_disable_rule_probability = 0; INSERT INTO xy VALUES (1, 2); SELECT 1 + 1; SELECT 100 INTO i; SELECT max(x) INTO i FROM xy; INSERT INTO xy VALUES (10, 10) RETURNING x INTO i; RETURN i; END $$ LANGUAGE PLpgSQL; ``` Informs #105252 Release note: None #### plpgsql: raise end-of-function error during runtime This commit moves the end-of-function error from compile-time to run-time, using a RAISE statement to throw the error. This more closely mirrors postgres behavior, where it's possible to define a function that is not guaranteed to terminate. Example: ``` CREATE OR REPLACE FUNCTION f(a INT, b INT) RETURNS INT AS $$ BEGIN IF a < b THEN RETURN a; END IF; END $$ LANGUAGE PLpgSQL; root@localhost:26257/defaultdb> SELECT f(1, 2); f ----- 1 (1 row) Time: 25ms total (execution 24ms / network 0ms) root@localhost:26257/defaultdb> SELECT f(2, 1); ERROR: control reached end of function without RETURN SQLSTATE: 2F005 ``` This commit also refactors some of the PLpgSQL routine-building logic to simplify building multiple body statements. Epic: CRDB-799 Release note: None #### plpgsql: add support for sql exec and INTO statements This patch adds execution support for executing SQL statements within PLpgSQL routines, both for the case when the result is discarded and when `INTO` syntax is used to assign the result to a set of target variables. The former case is handled by building the SQL statement into one of the first body statements of a routine; this captures the result-discarding behavior, but ensures any side effects still occur. The latter case is handled by aliasing the output columns of the SQL statement to the target variable names; this is done in the last body statement of the routine, since the result of the query is needed. Examples: ``` CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ BEGIN INSERT INTO xy VALUES (1000, 1000); DELETE FROM xy WHERE x = 1000; RETURN 0; END $$ LANGUAGE PLpgSQL; CREATE OR REPLACE FUNCTION f() RETURNS INT AS $$ DECLARE i INT; BEGIN SELECT x INTO i FROM xy ORDER BY x DESC; RETURN i; END $$ LANGUAGE PLpgSQL; ``` Fixes #105252 Release note (sql change): Added support for executing SQL statements directly within PLpgSQL routines. Note that this currently only applies to the subset of statements that can be executed within SQL UDFs, so `CREATE TABLE` is not supported, for example. INTO syntax is also supported, e.g. `SELECT * INTO a, b FROM xy;`. 108923: compose/compare: Add PG compat session variables r=chrisseto a=chrisseto Previously, `TestComposeCompare` would fail due to differences in how floats were cast to strings and the default behavior of NULL ordering. This commit sets PG compatibility session variables for the CockroachDB instances used in this test. This commit only fixes failures when: 1. A float is being cast to a string (`SELECT 1234567::FLOAT8::STRING`) 2. A `SELECT` contains an `ORDER BY` _without_ specifying how `NULLS` should be handled. There are still other known and unknown failures. Epic: None Informs: #99181 Co-authored-by: healthy-pod <ahmad@cockroachlabs.com> Co-authored-by: Drew Kimball <drewk@cockroachlabs.com> Co-authored-by: Chris Seto <chriskseto@gmail.com>
4 parents ceccd41 + 3b4e8f6 + b0034d1 + 7c307a8 commit 8004dcd

File tree

22 files changed

+1309
-308
lines changed

22 files changed

+1309
-308
lines changed

pkg/cli/mt_start_sql.go

Lines changed: 1 addition & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,6 @@ import (
1414
"context"
1515

1616
"github.com/cockroachdb/cockroach/pkg/cli/clierrorplus"
17-
"github.com/cockroachdb/cockroach/pkg/clusterversion"
1817
"github.com/cockroachdb/cockroach/pkg/server"
1918
"github.com/cockroachdb/cockroach/pkg/server/serverctl"
2019
"github.com/cockroachdb/cockroach/pkg/util/stop"
@@ -60,23 +59,7 @@ func runStartSQL(cmd *cobra.Command, args []string) error {
6059
const serverType redact.SafeString = "SQL server"
6160

6261
initConfig := func(ctx context.Context) error {
63-
if err := serverCfg.InitSQLServer(ctx); err != nil {
64-
return err
65-
}
66-
67-
// We need a value in the version setting prior to the update
68-
// coming from the system.settings table. This value must be valid
69-
// and compatible with the state of the tenant's keyspace.
70-
//
71-
// Since we don't know at which binary version the tenant
72-
// keyspace was initialized, we must be conservative and
73-
// assume it was created a long time ago; and that we may
74-
// have to run all known migrations since then. So initialize
75-
// the version setting to the minimum supported version.
76-
st := serverCfg.BaseConfig.Settings
77-
return clusterversion.Initialize(
78-
ctx, st.Version.BinaryMinSupportedVersion(), &st.SV,
79-
)
62+
return serverCfg.InitSQLServer(ctx)
8063
}
8164

8265
newServerFn := func(ctx context.Context, serverCfg server.Config, stopper *stop.Stopper) (serverctl.ServerStartupInterface, error) {

pkg/compose/compare/compare/compare_test.go

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -65,6 +65,8 @@ func TestCompare(t *testing.T) {
6565
addr: "postgresql://root@cockroach1:26257/postgres?sslmode=disable",
6666
init: []string{
6767
"SET CLUSTER SETTING cluster.organization = 'Cockroach Labs - Production Testing'",
68+
"SET extra_float_digits = 0", // For Postgres Compat when casting floats to strings.
69+
"SET null_ordered_last = true", // For Postgres Compat, see https://www.cockroachlabs.com/docs/stable/order-by#parameters
6870
fmt.Sprintf("SET CLUSTER SETTING enterprise.license = '%s'", license),
6971
"drop database if exists postgres",
7072
"create database postgres",
@@ -74,6 +76,8 @@ func TestCompare(t *testing.T) {
7476
addr: "postgresql://root@cockroach2:26257/postgres?sslmode=disable",
7577
init: []string{
7678
"SET CLUSTER SETTING cluster.organization = 'Cockroach Labs - Production Testing'",
79+
"SET extra_float_digits = 0", // For Postgres Compat when casting floats to strings.
80+
"SET null_ordered_last = true", // For Postgres Compat https://www.cockroachlabs.com/docs/stable/order-by#parameters
7781
fmt.Sprintf("SET CLUSTER SETTING enterprise.license = '%s'", license),
7882
"drop database if exists postgres",
7983
"create database postgres",

pkg/server/settingswatcher/settings_watcher.go

Lines changed: 16 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -339,15 +339,26 @@ func (s *SettingsWatcher) setLocked(
339339
// tenant, or because the new version <= old version).
340340
if key == versionSettingKey && !s.codec.ForSystemTenant() {
341341
var newVersion clusterversion.ClusterVersion
342-
oldVersion := s.settings.Version.ActiveVersion(ctx)
342+
oldVersion := s.settings.Version.ActiveVersionOrEmpty(ctx)
343343
if err := protoutil.Unmarshal([]byte(val.Value), &newVersion); err != nil {
344344
log.Warningf(ctx, "failed to set cluster version: %s", err.Error())
345345
} else if newVersion.LessEq(oldVersion.Version) {
346346
// Nothing to do.
347-
} else if err := s.settings.Version.SetActiveVersion(ctx, newVersion); err != nil {
348-
log.Warningf(ctx, "failed to set cluster version: %s", err.Error())
349-
} else if newVersion != oldVersion {
350-
log.Infof(ctx, "set cluster version from %v to: %v", oldVersion, newVersion)
347+
} else {
348+
// Check if cluster version setting is initialized. If it is empty then it is not
349+
// initialized.
350+
if oldVersion.Version.Equal(roachpb.Version{}) {
351+
// Cluster version setting not initialized.
352+
if err := clusterversion.Initialize(ctx, newVersion.Version, &s.settings.SV); err != nil {
353+
log.Fatalf(ctx, "failed to initialize cluster version setting: %s", err.Error())
354+
return
355+
}
356+
}
357+
if err := s.settings.Version.SetActiveVersion(ctx, newVersion); err != nil {
358+
log.Warningf(ctx, "failed to set cluster version: %s", err.Error())
359+
} else {
360+
log.Infof(ctx, "set cluster version from %v to: %v", oldVersion, newVersion)
361+
}
351362
}
352363
return
353364
}

pkg/server/testserver.go

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -1406,18 +1406,6 @@ func (ts *testServer) StartTenant(
14061406
if st == nil {
14071407
st = cluster.MakeTestingClusterSettings()
14081408
}
1409-
// Verify that the settings object that was passed in has
1410-
// initialized the version setting. This is pretty much necessary
1411-
// for secondary tenants. See the comments at the beginning of
1412-
// `runStartSQL()` in cli/mt_start_sql.go and
1413-
// `makeSharedProcessTenantServerConfig()` in
1414-
// server_controller_new_server.go.
1415-
//
1416-
// The version is initialized in MakeTestingClusterSettings(). This
1417-
// assertion is there to prevent inadvertent changes to
1418-
// MakeTestingClusterSettings() and as a guardrail for tests that
1419-
// pass a custom params.Settings.
1420-
clusterversion.AssertInitialized(ctx, &st.SV)
14211409

14221410
// Needed for backward-compat on crdb_internal.ranges{_no_leases}.
14231411
// Remove in v23.2.

0 commit comments

Comments
 (0)