Skip to content

Commit

Permalink
Merge #59194 #59201
Browse files Browse the repository at this point in the history
59194: kv: introduce a stopgap for lack of ReplicaState synchronization r=irfansharif a=irfansharif

There's a scary lack of synchronization around how we set the
ReplicaState for a given replica, and how we mark a replica as
"initialized". What this means is that very temporarily, it's possible
for the entry in Store.mu.replicas to be both "initialized" and have an
empty ReplicaState. This was an existing problem, but is now more likely
to bite us given the migrations infrastructure attempts to purge
outdated replicas at start up time (when replicas are being initialized,
and we're iterating through extan replicas in the Store.mu.replicas
map).

This issue has caused a bit of recent instability: #59180, #58489,
\#58523, and #58378. While we work on a more considered fix to the
problem (tracked in #58489), we can introduce stop the bleeding in the
interim (and unskip some tests).

Release note: None

59201:  sql: add telemetry for materialized views and set schema. r=otan a=RichardJCai

 sql: add telemetry for materialized views and set schema.

Release note: None

Resolves #57299 

Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: richardjcai <caioftherichard@gmail.com>
  • Loading branch information
3 people committed Jan 20, 2021
3 parents ffbd3bd + f47e1f3 + ea15a7a commit e9d2cc1
Show file tree
Hide file tree
Showing 15 changed files with 163 additions and 13 deletions.
1 change: 0 additions & 1 deletion pkg/cmd/roachtest/acceptance.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,6 @@ func registerAcceptance(r *testRegistry) {
// to head after 19.2 fails.
minVersion: "v19.2.0",
timeout: 30 * time.Minute,
skip: "https://github.com/cockroachdb/cockroach/issues/58489",
},
}
tags := []string{"default", "quick"}
Expand Down
11 changes: 7 additions & 4 deletions pkg/cmd/roachtest/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,9 @@ func runVersionUpgrade(ctx context.Context, t *test, c *cluster, buildVersion ve

testFeaturesStep := versionUpgradeTestFeatures.step(c.All())
schemaChangeStep := runSchemaChangeWorkloadStep(c.All().randNode()[0], 10 /* maxOps */, 2 /* concurrency */)
// TODO(irfansharif): All schema change instances were commented out while
// of #58489 is being addressed.
_ = schemaChangeStep
backupStep := func(ctx context.Context, t *test, u *versionUpgradeTest) {
// This check was introduced for the system.tenants table and the associated
// changes to full-cluster backup to include tenants. It mostly wants to
Expand Down Expand Up @@ -151,26 +154,26 @@ func runVersionUpgrade(ctx context.Context, t *test, c *cluster, buildVersion ve
testFeaturesStep,
// Run a quick schemachange workload in between each upgrade.
// The maxOps is 10 to keep the test runtime under 1-2 minutes.
schemaChangeStep,
// schemaChangeStep,
backupStep,
// Roll back again. Note that bad things would happen if the cluster had
// ignored our request to not auto-upgrade. The `autoupgrade` roachtest
// exercises this in more detail, so here we just rely on things working
// as they ought to.
binaryUpgradeStep(c.All(), predecessorVersion),
testFeaturesStep,
schemaChangeStep,
// schemaChangeStep,
backupStep,
// Roll nodes forward, this time allowing them to upgrade, and waiting
// for it to happen.
binaryUpgradeStep(c.All(), ""),
allowAutoUpgradeStep(1),
testFeaturesStep,
schemaChangeStep,
// schemaChangeStep,
backupStep,
waitForUpgradeStep(c.All()),
testFeaturesStep,
schemaChangeStep,
// schemaChangeStep,
backupStep,
)

Expand Down
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -802,6 +802,11 @@ func (r *Replica) GetGCThreshold() hlc.Timestamp {
func (r *Replica) Version() roachpb.Version {
r.mu.RLock()
defer r.mu.RUnlock()

if r.mu.state.Version == nil {
// TODO(irfansharif): This is a stop-gap for #58523.
return roachpb.Version{}
}
return *r.mu.state.Version
}

Expand Down
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -2810,6 +2810,10 @@ func (s *Store) PurgeOutdatedReplicas(ctx context.Context, version roachpb.Versi
qp := quotapool.NewIntPool("purge-outdated-replicas", 50)
g := ctxgroup.WithContext(ctx)
s.VisitReplicas(func(repl *Replica) (wantMore bool) {
if (repl.Version() == roachpb.Version{}) {
// TODO(irfansharif): This is a stop gap for #58523.
return true
}
if !repl.Version().Less(version) {
// Nothing to do here.
return true
Expand Down
2 changes: 0 additions & 2 deletions pkg/server/migration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

Expand Down Expand Up @@ -201,7 +200,6 @@ func TestBumpClusterVersion(t *testing.T) {

func TestMigrationPurgeOutdatedReplicas(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 59180)

const numStores = 3
var storeSpecs []base.StoreSpec
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/alter_table_set_schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package sql
import (
"context"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
Expand Down Expand Up @@ -92,6 +93,7 @@ func (p *planner) AlterTableSetSchema(
}

func (n *alterTableSetSchemaNode) startExec(params runParams) error {
telemetry.Inc(n.n.TelemetryCounter())
ctx := params.ctx
p := params.p
tableDesc := n.tableDesc
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/create_view.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,10 +60,13 @@ type createViewNode struct {
func (n *createViewNode) ReadingOwnWrites() {}

func (n *createViewNode) startExec(params runParams) error {
tableType := tree.GetTableType(
false /* isSequence */, true /* isView */, n.materialized,
)
if n.replace {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("or_replace_view"))
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter(fmt.Sprintf("or_replace_%s", tableType)))
} else {
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter("view"))
telemetry.Inc(sqltelemetry.SchemaChangeCreateCounter(tableType))
}

viewName := n.viewName.Object()
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/drop_view.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -91,7 +90,7 @@ func (p *planner) DropView(ctx context.Context, n *tree.DropView) (planNode, err
func (n *dropViewNode) ReadingOwnWrites() {}

func (n *dropViewNode) startExec(params runParams) error {
telemetry.Inc(sqltelemetry.SchemaChangeDropCounter("view"))
telemetry.Inc(n.n.TelemetryCounter())

ctx := params.ctx
for _, toDel := range n.td {
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/refresh_materialized_view.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package sql
import (
"context"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
Expand Down Expand Up @@ -58,6 +59,8 @@ func (n *refreshMaterializedViewNode) startExec(params runParams) error {
// results of the view query into the new set of indexes, and then change the
// set of indexes over to the new set of indexes atomically.

telemetry.Inc(n.n.TelemetryCounter())

// Inform the user that CONCURRENTLY is not needed.
if n.n.Concurrently {
params.p.BufferClientNotice(
Expand Down
35 changes: 33 additions & 2 deletions pkg/sql/sem/tree/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -600,6 +600,14 @@ func (node *AlterTableSetSchema) Format(ctx *FmtCtx) {
ctx.FormatNode(&node.Schema)
}

// TelemetryCounter returns the telemetry counter to increment
// when this command is used.
func (node *AlterTableSetSchema) TelemetryCounter() telemetry.Counter {
return sqltelemetry.SchemaChangeAlterCounterWithExtra(
GetTableType(node.IsSequence, node.IsView, node.IsMaterialized),
"set_schema")
}

// AlterTableOwner represents an ALTER TABLE OWNER TO command.
type AlterTableOwner struct {
Name *UnresolvedObjectName
Expand All @@ -612,9 +620,13 @@ type AlterTableOwner struct {
IsSequence bool
}

// TelemetryCounter implements the AlterTableCmd interface.
// TelemetryCounter returns the telemetry counter to increment
// when this command is used.
func (node *AlterTableOwner) TelemetryCounter() telemetry.Counter {
return sqltelemetry.SchemaChangeAlterCounterWithExtra("table", "owner to")
return sqltelemetry.SchemaChangeAlterCounterWithExtra(
GetTableType(node.IsSequence, node.IsView, node.IsMaterialized),
"owner_to",
)
}

// Format implements the NodeFormatter interface.
Expand All @@ -637,3 +649,22 @@ func (node *AlterTableOwner) Format(ctx *FmtCtx) {
ctx.WriteString(" OWNER TO ")
ctx.FormatUsername(node.Owner)
}

// GetTableType returns a string representing the type of table the command
// is operating on.
// It is assumed if the table is not a sequence or a view, then it is a
// regular table.
func GetTableType(isSequence bool, isView bool, isMaterialized bool) string {
tableType := "table"
if isSequence {
tableType = "sequence"
} else if isView {
if isMaterialized {
tableType = "materialized_view"
} else {
tableType = "view"
}
}

return tableType
}
8 changes: 8 additions & 0 deletions pkg/sql/sem/tree/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,12 @@ import (
"strings"

"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/pretty"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -1786,6 +1788,12 @@ type RefreshMaterializedView struct {
RefreshDataOption RefreshDataOption
}

// TelemetryCounter returns the telemetry counter to increment
// when this command is used.
func (node *RefreshMaterializedView) TelemetryCounter() telemetry.Counter {
return sqltelemetry.SchemaRefreshMaterializedView
}

// RefreshDataOption corresponds to arguments for the REFRESH MATERIALIZED VIEW
// statement.
type RefreshDataOption int
Expand Down
14 changes: 14 additions & 0 deletions pkg/sql/sem/tree/drop.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,11 @@

package tree

import (
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
)

// DropBehavior represents options for dropping schema elements.
type DropBehavior int

Expand Down Expand Up @@ -128,6 +133,15 @@ func (node *DropView) Format(ctx *FmtCtx) {
}
}

// TelemetryCounter returns the telemetry counter to increment
// when this command is used.
func (node *DropView) TelemetryCounter() telemetry.Counter {
return sqltelemetry.SchemaChangeDropCounter(
GetTableType(
false /* isSequence */, true, /* isView */
node.IsMaterialized))
}

// DropSequence represents a DROP SEQUENCE statement.
type DropSequence struct {
Names TableNames
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/sqltelemetry/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -142,3 +142,7 @@ var SecondaryIndexColumnFamiliesCounter = telemetry.GetCounterOnce("sql.schema.s
// CreateUnloggedTableCounter is a counter that is incremented every time an unlogged
// table is created.
var CreateUnloggedTableCounter = telemetry.GetCounterOnce("sql.schema.create_unlogged_table")

// SchemaRefreshMaterializedView is to be incremented every time a materialized
// view is refreshed.
var SchemaRefreshMaterializedView = telemetry.GetCounterOnce("sql.schema.refresh_materialized_view")
38 changes: 38 additions & 0 deletions pkg/sql/testdata/telemetry/materialized_view
Original file line number Diff line number Diff line change
@@ -0,0 +1,38 @@
# This file contains telemetry tests for sql.schema.*_materialized_view.* counters.

feature-allowlist
sql.schema.create_materialized_view
sql.schema.alter_materialized_view.owner_to
sql.schema.alter_materialized_view.set_schema
sql.schema.refresh_materialized_view
sql.schema.drop_materialized_view
----

feature-usage
CREATE MATERIALIZED VIEW mvx AS SELECT 1
----
sql.schema.create_materialized_view

feature-usage
REFRESH MATERIALIZED VIEW mvx
----
sql.schema.refresh_materialized_view

feature-usage
ALTER MATERIALIZED VIEW mvx OWNER TO root
----
sql.schema.alter_materialized_view.owner_to

exec
CREATE SCHEMA ss
----

feature-usage
ALTER MATERIALIZED VIEW mvx SET SCHEMA ss
----
sql.schema.alter_materialized_view.set_schema

feature-usage
DROP MATERIALIZED VIEW ss.mvx
----
sql.schema.drop_materialized_view
39 changes: 39 additions & 0 deletions pkg/sql/testdata/telemetry/set_schema
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
# This file contains telemetry tests for sql.schema.alter_*.set_schema counters.

feature-allowlist
sql.schema.alter_table.set_schema
sql.schema.alter_sequence.set_schema
sql.schema.alter_view.set_schema
sql.schema.alter_materialized_view.set_schema
----

exec
CREATE SCHEMA ss
----

exec
CREATE TABLE t();
CREATE SEQUENCE s;
CREATE VIEW vx AS SELECT 1;
CREATE MATERIALIZED VIEW mvx AS SELECT 1
----

feature-usage
ALTER TABLE t SET SCHEMA ss
----
sql.schema.alter_table.set_schema

feature-usage
ALTER SEQUENCE s SET SCHEMA ss
----
sql.schema.alter_sequence.set_schema

feature-usage
ALTER VIEW vx SET SCHEMA ss
----
sql.schema.alter_view.set_schema

feature-usage
ALTER MATERIALIZED VIEW mvx SET SCHEMA ss
----
sql.schema.alter_materialized_view.set_schema

0 comments on commit e9d2cc1

Please sign in to comment.