Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96112: ui: add search, filter and time picker on index details r=maryliag a=maryliag

Fixes #93087

This commit adds search, filters and time picker for the list of fingerprints most used per index, on the Index Details page.

It also limits to 20 fingerprints until we can make improvements to use pagination on our calls with
sql-over-http.

With no filters
<img width="1669" alt="Screen Shot 2023-01-27 at 1 37 35 PM" src="https://user-images.githubusercontent.com/1017486/215167830-bd119e13-a875-49bd-9c8d-305230dd4b01.png">

With filters
<img width="1508" alt="Screen Shot 2023-01-27 at 1 37 49 PM" src="https://user-images.githubusercontent.com/1017486/215167846-c668d79a-7a59-420e-b96c-2eff6a50aca6.png">


https://www.loom.com/share/d4129452593a40198902a5f2539d8568

Release note (ui change): Add search, filter and time picker for the list of most used statement fingerprints on the Index Details page.

96117: schemachanger: improve state management r=postamar a=postamar

This commit endows the scpb.CurrentState with the set of element statuses which are in force at the beginning of the statement transaction. These are then used when planning the pre-commit reset stage.

This commit also rewrites the target-setting logic in the scbuild package in a more explicit manner with extra commentary.

This commit does not change any functional behavior in production.

Informs #88294.

Release note: None

96232: sql: move copy_in_test to pkg/sql/copy to have all copy tests together r=cucaroach a=cucaroach

Epic: CRDB-18892
Informs: #91831
Release note: None


96272: backupccl,changefeedccl: properly display name if resource exists r=msbutler,HonoreDB a=renatolabs

Previously, we were passing a `*string` to a string formatting function (`pgnotice.Newf`) with the `%q` verb. This leads to messages being displayed to the user that look like:

```
NOTICE: schedule %!q(*string=0xc006b324e0) already exists, skipping
```

This properly dereferences the pointer before printing.

Epic: none

Release note: None

Co-authored-by: maryliag <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: Tommy Reilly <[email protected]>
Co-authored-by: Renato Costa <[email protected]>
  • Loading branch information
5 people committed Jan 31, 2023
5 parents e4a4340 + 0b405dc + 1266859 + 9894417 + b4b2835 commit e402d82
Show file tree
Hide file tree
Showing 34 changed files with 724 additions and 334 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ func doCreateBackupSchedules(

if exists {
p.BufferClientNotice(ctx,
pgnotice.Newf("schedule %q already exists, skipping", eval.scheduleLabel),
pgnotice.Newf("schedule %q already exists, skipping", *eval.scheduleLabel),
)
return nil
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/scheduled_changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -562,7 +562,7 @@ func doCreateChangefeedSchedule(

if exists {
p.BufferClientNotice(ctx,
pgnotice.Newf("schedule %q already exists, skipping", spec.scheduleLabel),
pgnotice.Newf("schedule %q already exists, skipping", *spec.scheduleLabel),
)
return nil
}
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -580,7 +580,6 @@ go_test(
"conn_executor_savepoints_test.go",
"conn_executor_test.go",
"conn_io_test.go",
"copy_in_test.go",
"copy_test.go",
"crdb_internal_test.go",
"create_function_test.go",
Expand Down Expand Up @@ -826,8 +825,6 @@ go_test(
"//pkg/util/shuffle",
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeofday",
"//pkg/util/timetz",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/tracing/tracingpb",
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/copy/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_test")
go_test(
name = "copy_test",
srcs = [
"copy_in_test.go",
"copy_test.go",
"main_test.go",
],
Expand All @@ -12,22 +13,34 @@ go_test(
deps = [
"//pkg/base",
"//pkg/cli/clisqlclient",
"//pkg/roachpb",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/security/username",
"//pkg/server",
"//pkg/settings/cluster",
"//pkg/sql",
"//pkg/sql/randgen",
"//pkg/sql/sem/tree",
"//pkg/sql/sqltestutils",
"//pkg/sql/tests",
"//pkg/sql/types",
"//pkg/testutils",
"//pkg/testutils/datapathutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/sqlutils",
"//pkg/testutils/testcluster",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/randutil",
"//pkg/util/timeofday",
"//pkg/util/timetz",
"@com_github_cockroachdb_apd_v3//:apd",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_jackc_pgtype//:pgtype",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_lib_pq//:pq",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
],
)
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/copy_in_test.go → pkg/sql/copy/copy_in_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql_test
package copy

import (
"context"
Expand All @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltestutils"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -778,7 +779,7 @@ func TestCopyInReleasesLeases(t *testing.T) {
s.ServingSQLAddr(), t.Name(), url.UserPassword("foo", "testabc"),
false /* withClientCerts */)
defer cleanupFn()
conn, err := pgxConn(t, userURL)
conn, err := sqltestutils.PGXConn(t, userURL)
require.NoError(t, err)

rowsAffected, err := conn.CopyFrom(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/copy/copy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package copy_test
package copy

import (
"context"
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/copy/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package copy_test
package copy

import (
"os"
Expand Down
7 changes: 3 additions & 4 deletions pkg/sql/schemachanger/corpus/corpus.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ func (cc *Collector) GetBeforeStage(
corpusName, corpusPrefix := makeCorpusName(parentName, t.Name(), p.Statements, stageIdx)
entry := &scpb.CorpusState{
Name: corpusName,
Status: p.Current,
Status: p.Initial,
TargetState: &p.TargetState,
InRollback: p.InRollback,
Revertible: p.Revertible,
Expand Down Expand Up @@ -297,12 +297,11 @@ func (cr *Reader) GetNumEntries() int {
// state.
func (cr *Reader) GetCorpus(idx int) (name string, state *scpb.CurrentState) {
corpus := cr.diskState.CorpusArray[idx]
name = corpus.Name
state = &scpb.CurrentState{
return corpus.Name, &scpb.CurrentState{
TargetState: *corpus.TargetState,
Initial: corpus.Status,
Current: corpus.Status,
InRollback: corpus.InRollback,
Revertible: corpus.Revertible,
}
return name, state
}
85 changes: 47 additions & 38 deletions pkg/sql/schemachanger/scbuild/build.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,21 +29,26 @@ import (
"github.com/cockroachdb/redact"
)

// Build constructs a new state from an initial state and a statement.
// Build constructs a new state from an incumbent state and a statement.
//
// The function takes an AST for a DDL statement and constructs targets
// which represent schema changes to be performed.
// which represent schema changes to be performed. The incumbent state
// is the schema changer state for the statement transaction, which reached
// its present value from preceding calls to Build which were followed by the
// execution of their corresponding statement phase stages. In other words,
// the incumbent state encodes the schema change such as it has been defined
// prior to this call, along with any in-transaction side effects.
func Build(
ctx context.Context, dependencies Dependencies, initial scpb.CurrentState, n tree.Statement,
ctx context.Context, dependencies Dependencies, incumbent scpb.CurrentState, n tree.Statement,
) (_ scpb.CurrentState, err error) {
defer scerrors.StartEventf(
ctx,
"building declarative schema change targets for %s",
redact.Safe(n.StatementTag()),
).HandlePanicAndLogError(ctx, &err)
initial = initial.DeepCopy()
bs := newBuilderState(ctx, dependencies, initial)
els := newEventLogState(dependencies, initial, n)
incumbent = incumbent.DeepCopy()
bs := newBuilderState(ctx, dependencies, incumbent)
els := newEventLogState(dependencies, incumbent, n)
// TODO(fqazi): The optimizer can end up already modifying the statement above
// to fully resolve names. We need to take this into account for CTAS/CREATE
// VIEW statements.
Expand All @@ -61,13 +66,15 @@ func Build(
}
scbuildstmt.Process(b, an.GetStatement())
an.ValidateAnnotations()
els.statements[len(els.statements)-1].RedactedStatement = string(
currentStatementID := uint32(len(els.statements) - 1)
els.statements[currentStatementID].RedactedStatement = string(
dependencies.AstFormatter().FormatAstAsRedactableString(an.GetStatement(), &an.annotation))
ts := scpb.TargetState{
Targets: make([]scpb.Target, 0, len(bs.output)),
Statements: els.statements,
Authorization: els.authorization,
}
initial := make([]scpb.Status, 0, len(bs.output))
current := make([]scpb.Status, 0, len(bs.output))
version := dependencies.ClusterSettings().Version.ActiveVersion(ctx)
withLogEvent := make([]scpb.Target, 0, len(bs.output))
Expand All @@ -82,24 +89,9 @@ func Build(
// cluster version.
continue
}
if e.previous == scpb.InvalidTarget {
// The target was newly-defined by this build.
if e.target.Status() == e.current {
// Discard it if it's already fulfilled.
continue
}
} else if e.previous.InitialStatus() == scpb.Status_ABSENT && e.target == scpb.ToAbsent {
// The target was defined as an adding target by a previous build.
// This build redefines it as a dropping target.
// As far as the schema change is concerned, this is a no-op, so we
// discard this target.
// TODO(postamar): this might be too crude, the target's absence might
// cause necessary statement-phase ops to be omitted, leading to
// incorrect in-txn behaviour.
continue
}
t := scpb.MakeTarget(e.target, e.element, &e.metadata)
ts.Targets = append(ts.Targets, t)
initial = append(initial, e.initial)
current = append(current, e.current)
if e.withLogEvent {
withLogEvent = append(withLogEvent, t)
Expand All @@ -116,7 +108,11 @@ func Build(
})
// Write to event log and return.
logEvents(b, ts, withLogEvent)
return scpb.CurrentState{TargetState: ts, Current: current}, nil
return scpb.CurrentState{
TargetState: ts,
Initial: initial,
Current: current,
}, nil
}

// CheckIfSupported returns if a statement is fully supported by the declarative
Expand All @@ -137,12 +133,11 @@ type (
type elementState struct {
// element is the element which identifies this structure.
element scpb.Element
// current is the current status of the element.
current scpb.Status
// target is the target to be fulfilled by the element status, if applicable,
// while previous is the target for this element as defined by an earlier call
// to scbuild.Build in the same transaction, if applicable.
previous, target scpb.TargetStatus
// current is the current status of the element;
// initial is the status of the element at the beginning of the transaction.
initial, current scpb.Status
// target indicates the status to be fulfilled by the element.
target scpb.TargetStatus
// metadata contains the target metadata to store in the resulting
// scpb.TargetState produced by the current call to scbuild.Build.
metadata scpb.TargetMetadata
Expand Down Expand Up @@ -193,7 +188,9 @@ type cachedDesc struct {
}

// newBuilderState constructs a builderState.
func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentState) *builderState {
func newBuilderState(
ctx context.Context, d Dependencies, incumbent scpb.CurrentState,
) *builderState {
bs := builderState{
ctx: ctx,
clusterSettings: d.ClusterSettings(),
Expand All @@ -203,7 +200,7 @@ func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentSt
tr: d.TableReader(),
auth: d.AuthorizationAccessor(),
createPartCCL: d.IndexPartitioningCCLCallback(),
output: make([]elementState, 0, len(initial.Current)),
output: make([]elementState, 0, len(incumbent.Current)),
descCache: make(map[catid.DescID]*cachedDesc),
tempSchemas: make(map[catid.DescID]catalog.SchemaDescriptor),
commentGetter: d.DescriptorCommentGetter(),
Expand All @@ -214,12 +211,22 @@ func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentSt
if err != nil {
panic(err)
}
for _, t := range initial.TargetState.Targets {
for _, t := range incumbent.TargetState.Targets {
bs.ensureDescriptor(screl.GetDescID(t.Element()))
}
for i, t := range initial.TargetState.Targets {
ts := scpb.AsTargetStatus(t.TargetStatus)
bs.ensure(t.Element(), initial.Current[i], ts, ts, t.Metadata)
for i, t := range incumbent.TargetState.Targets {
src := elementState{
element: t.Element(),
initial: incumbent.Initial[i],
current: incumbent.Current[i],
target: scpb.AsTargetStatus(t.TargetStatus),
metadata: t.Metadata,
}
if dst := bs.getExistingElementState(src.element); dst != nil {
*dst = src
} else {
bs.addNewElementState(src)
}
}
return &bs
}
Expand All @@ -244,8 +251,10 @@ type eventLogState struct {
}

// newEventLogState constructs an eventLogState.
func newEventLogState(d Dependencies, initial scpb.CurrentState, n tree.Statement) *eventLogState {
stmts := initial.Statements
func newEventLogState(
d Dependencies, incumbent scpb.CurrentState, n tree.Statement,
) *eventLogState {
stmts := incumbent.Statements
els := eventLogState{
statements: append(stmts, scpb.Statement{
Statement: n.String(),
Expand Down
Loading

0 comments on commit e402d82

Please sign in to comment.