Skip to content

Commit

Permalink
migrations: populate initial version setting
Browse files Browse the repository at this point in the history
This uses the previously added bootstrap version which is populated when a
cluster running >1.0 is bootstrapped to populate the settings table. At the
same time, prevent `SET CLUSTER SETTING version = 'x'` from working until
that migration has run.

This solves one remaining headache for [version migrations][1] by giving it
authoritative information on the currently running cluster version during
upgrades.

Fixes cockroachdb#17389.

[1]: cockroachdb#16977
  • Loading branch information
tbg committed Aug 16, 2017
1 parent 2523ba5 commit eddc62c
Show file tree
Hide file tree
Showing 8 changed files with 94 additions and 54 deletions.
58 changes: 58 additions & 0 deletions pkg/migration/sqlmigrations/migrations.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
package sqlmigrations

import (
"fmt"
"time"

"golang.org/x/net/context"
Expand All @@ -24,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
Expand Down Expand Up @@ -77,6 +79,10 @@ var backwardCompatibleMigrations = []migrationDescriptor{
// table ID by 4 (3 IDs were reserved for non-table entities).
newRanges: 4,
},
{
name: "populate initial version cluster setting table entry",
workFn: populateVersionSetting,
},
}

// migrationDescriptor describes a single migration hook that's used to modify
Expand Down Expand Up @@ -405,6 +411,58 @@ func optInToDiagnosticsStatReporting(ctx context.Context, r runner) error {
return err
}

func populateVersionSetting(ctx context.Context, r runner) error {
var v roachpb.Version
if err := r.db.Txn(ctx, func(ctx context.Context, txn *client.Txn) error {
return txn.GetProto(ctx, keys.BootstrapVersionKey, &v)
}); err != nil {
return err
}
if v == (roachpb.Version{}) {
// The cluster was bootstrapped at v1.0 (or even earlier), so make that
// the version.
v = cluster.VersionBase
}

b, err := (&cluster.ClusterVersion{
MinimumVersion: v,
UseVersion: v,
}).Marshal()
if err != nil {
return errors.Wrap(err, "while marshaling version")
}

// System tables can only be modified by a privileged internal user.
session := r.newRootSession(ctx)
defer session.Finish(r.sqlExecutor)

// Add a ON CONFLICT DO NOTHING to avoid changing an existing version.
// Again, this can happen if the migration doesn't run to completion
// (overwriting also seems reasonable, but what for).
// We don't allow users to perform version changes until we have run
// the insert below.
if res, err := r.sqlExecutor.ExecuteStatementsBuffered(
session,
fmt.Sprintf(`INSERT INTO system.settings (name, value, "lastUpdated", "valueType") VALUES ('version', x'%x', NOW(), 'm') ON CONFLICT(name) DO NOTHING`, b),
nil, 1,
); err == nil {
res.Close(ctx)
} else if err != nil {
return err
}

// TODO(tschottdorf): revisit after https://github.com/cockroachdb/cockroach/pull/17591.
if res, err := r.sqlExecutor.ExecuteStatementsBuffered(
session,
fmt.Sprintf("SET CLUSTER SETTING version = '%s'", v.String()),
nil, 1); err == nil {
res.Close(ctx)
} else if err != nil {
return err
}
return nil
}

// repopulateViewDeps recomputes the dependencies of all views, as
// they might not have been computed properly previously.
// (#17269 #17306)
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -750,7 +750,7 @@ func TestAdminAPIEvents(t *testing.T) {
{sql.EventLogCreateDatabase, false, 0, 1},
{sql.EventLogDropTable, false, 0, 2},
{sql.EventLogCreateTable, false, 0, 3},
{sql.EventLogSetClusterSetting, false, 0, 2},
{sql.EventLogSetClusterSetting, false, 0, 3},
{sql.EventLogCreateTable, true, 0, 3},
{sql.EventLogCreateTable, true, -1, 3},
{sql.EventLogCreateTable, true, 2, 2},
Expand Down
28 changes: 6 additions & 22 deletions pkg/server/version_cluster_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,11 +128,6 @@ func TestClusterVersionUpgrade1_0To1_1(t *testing.T) {
tc := setupMixedCluster(t, bootstrapVersion, versions)
defer tc.Stopper().Stop(ctx)

// TODO(tschottdorf): once the migration in #17389 is in place, verify that
// there's a suitable entry in the `system.settings` table for a freshly
// bootstrapped cluster (at 1.1). It's acceptable that an 1.0 cluster (as
// this one) doesn't write one.

for i := 0; i < tc.NumServers(); i++ {
if exp, version := bootstrapVersion.MinimumVersion.String(), tc.getVersionFromShow(i); version != exp {
t.Fatalf("%d: incorrect version %s (wanted %s)", i, version, exp)
Expand All @@ -150,6 +145,10 @@ func TestClusterVersionUpgrade1_0To1_1(t *testing.T) {
if isActive := v.IsActive(newVersion); isActive != wantActive {
t.Fatalf("%d: v%s active=%t (wanted %t)", i, newVersion, isActive, wantActive)
}

if tableV, curV := tc.getVersionFromSelect(i), v.Version().MinimumVersion.String(); tableV != curV {
t.Fatalf("%d: read v%s from table, v%s from setting", i, tableV, curV)
}
}

exp := newVersion.String()
Expand Down Expand Up @@ -229,23 +228,8 @@ func TestClusterVersionBootstrapStrict(t *testing.T) {
t.Fatalf("%d: incorrect version %s (wanted %s)", i, version, exp)
}

// TODO(tschottdorf): this documents the oddity that exists at
// the moment, see
// https://github.com/cockroachdb/cockroach/issues/17389.
//
// In short, there's nothing that initially populates the
// settings table as this is difficult to do during
// bootstrapping, and using a sql migration is awkward since
// we'd need a new migration for each release of CockroachDB and
// must not have users restart into a newer version immediately
// after bootstrap.
//
// Details here are TBD and should be updated with the outcome
// of the aforementioned issue.
brokenExp := "" // in a better world, would have brokenExp == exp

if version := tc.getVersionFromSelect(i); version != brokenExp {
t.Fatalf("%d: incorrect version %q (wanted %s)", i, version, brokenExp)
if version := tc.getVersionFromSelect(i); version != exp {
t.Fatalf("%d: incorrect version %q (wanted %s)", i, version, exp)
}
}
}()
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ SET CLUSTER SETTING kv.allocator.load_based_lease_rebalancing.enabled = DEFAULT
query IIT
SELECT "targetID", "reportingID", "info"
FROM system.eventlog
WHERE "eventType" = 'set_cluster_setting'
WHERE "eventType" = 'set_cluster_setting' AND info NOT LIKE '%version%'
ORDER BY "timestamp"
----
0 1 {"SettingName":"diagnostics.reporting.enabled","Value":"true","User":"node"}
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/system
Original file line number Diff line number Diff line change
Expand Up @@ -332,16 +332,19 @@ GRANT ALL ON system.lease TO root
statement ok
GRANT ALL ON system.lease TO testuser

# NB: the "order by" is necessary or this test is flaky under DistSQL.
# This is somewhat surprising.
query T
select name from system.settings
select name from system.settings order by name
----
diagnostics.reporting.enabled
version

statement ok
INSERT INTO system.settings (name, value) VALUES ('somesetting', 'somevalue')

query TT
select name, value from system.settings order by name
select name, value from system.settings where name != 'version' order by name
----
diagnostics.reporting.enabled true
somesetting somevalue
Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/metric_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,8 +76,9 @@ func TestQueryCounts(t *testing.T) {
// Initialize accum while accounting for system migrations that may have run
// DDL statements.
accum := queryCounter{
ddlCount: s.MustGetSQLCounter(sql.MetaDdl.Name),
miscCount: s.MustGetSQLCounter(sql.MetaMisc.Name),
insertCount: 1, // version setting population migration
ddlCount: s.MustGetSQLCounter(sql.MetaDdl.Name),
miscCount: s.MustGetSQLCounter(sql.MetaMisc.Name),
}

for _, tc := range testcases {
Expand Down Expand Up @@ -186,7 +187,7 @@ func TestAbortCountConflictingWrites(t *testing.T) {
if err := checkCounterEQ(s, sql.MetaTxnCommit, 0); err != nil {
t.Error(err)
}
if err := checkCounterEQ(s, sql.MetaInsert, 1); err != nil {
if err := checkCounterEQ(s, sql.MetaInsert, 2); err != nil {
t.Error(err)
}
}
Expand Down
33 changes: 13 additions & 20 deletions pkg/sql/set.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (

"github.com/cockroachdb/apd"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
Expand Down Expand Up @@ -183,30 +182,24 @@ func (p *planner) toSettingString(
if err != nil {
return "", err
}
var prevRawVal []byte
if len(datums) != 0 {
dStr, ok := datums[0].(*parser.DString)
if !ok {
return "", errors.New("the existing value is not a string")
}
prevRawVal = []byte(string(*dStr))
} else {
// If no entry is present, treat like v1.0.
// This is subject to change. See
// https://github.com/cockroachdb/cockroach/issues/17389
var err error
prevRawVal, err = (&cluster.ClusterVersion{
MinimumVersion: cluster.VersionBase,
UseVersion: cluster.VersionBase,
}).Marshal()
if err != nil {
return "", err
}
if len(datums) == 0 {
// There is a SQL migration which adds this value. If it
// hasn't run yet, we can't update the version as we don't
// have good enough information about the current cluster
// version.
return "", errors.New("no persisted cluster version found, please retry later")
}

dStr, ok := datums[0].(*parser.DString)
if !ok {
return "", errors.New("the existing value is not a string")
}
prevRawVal := []byte(string(*dStr))
newBytes, _, err := setting.Validate(prevRawVal, (*string)(s))
if err != nil {
return "", err
}

return string(newBytes), nil
}
return "", errors.Errorf("cannot use %s %T value for string setting", d.ResolvedType(), d)
Expand Down
11 changes: 6 additions & 5 deletions pkg/sql/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,11 +104,12 @@ func (p *planner) showClusterSetting(ctx context.Context, name string) (planNode
prevRawVal = []byte(string(*dStr))
}
// Note that if no entry is found, we pretend that an entry
// exists which is the version used for the running binary.
// This is a bit awkward, but we have no good way of ensuring
// that this entry is written at cluster bootstrap time.
// Subject to change. See
// https://github.com/cockroachdb/cockroach/issues/17389
// exists which is the version used for the running binary. This
// may not be 100.00% correct, but it will do. The input is
// checked more thoroughly when a user tries to change the
// value, and the corresponding sql migration that makes sure
// the above select finds something usually runs pretty quickly
// when the cluster is bootstrapped.
_, obj, err := s.Validate(prevRawVal, nil)
if err != nil {
return nil, errors.Errorf("unable to read existing value: %s", err)
Expand Down

0 comments on commit eddc62c

Please sign in to comment.