Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
71246: sql: remove InternalExecutor from EvalCtx r=rafiss a=RichardJCai

Minor refactor to remove duplicate InternalExecutor definition
on EvalCtx and in the tree package.

We already have 2 interface declarations for InternalExecutor
in sql and sqlutil, this refactor hopefully makes the dependencies
a little more clear.

Resolves #60507

72627: ui: unify cluster-ui stores for Statement and Transaction Page r=maryliag a=Azhng

Previously, on CC Conosle, Statement Page and Transaction Page were backed by
their own Redux stores and their own Redux Sagas. However, both Statement
and Transaction Page were backed by a single API endpoint, namely
`/_status/statements`. This resulted in unnecessary API calls and bugs
like #72009, which was caused by Reset SQL Stats sagas failing to
reset the Transaction Page Redux store.

This commit unifies the following Redux store and sagas into a single
store / sagas:
1. Statement Page store / sagas
2. Transaciton Page store / sagas
3. Reset SQL Stats store / sagas

This greatly removed the code duplication and test duplication and simplify
the logic. Statement Page and Transaction Page can reuse the same store
by their own selectors.

Resolves #72009

Release note: None


CC Console: 

https://user-images.githubusercontent.com/9267198/141212289-00c5196b-87c0-4bae-a163-3542e1a2b6a0.mov


DB Console: 

https://user-images.githubusercontent.com/9267198/141048060-482d23e6-dc77-4868-98dd-e209d0965cf9.mov

73003: ui: save filters on cache for Transactions page r=maryliag a=maryliag

Previously, a sort selection was not maintained when
the page change (e.g. changing tabs/pages).
This commits saves the selected value to be used.

Partially adresses #71851

Release note: None

73090: tabledesc: forbid computed columns from having DEFAULT expressions r=postamar a=postamar

Previously, we didn't have a table descriptor validation check to ensure
that computed columns didn't also have DEFAULT expressions. It is
therefore possible that clusters and cluster backups exist in production
which contain tables with such columns. This commit therefore fixes this
bug by adding the missing validation check, and by removing default
expressions from computed columns when necessary after deserializing
a descriptor protobuf.

Fixes #72881.

Release note (sql change, bug fix): fixes a bug which allowed computed
columns to also have DEFAULT expressions.

73125: opt: prevent creation of invalid streaming set operations r=rytaft a=rytaft

Prior to this commit, it was possible to generate a streaming
set operation with an empty ordering, due to using interesting
orderings involving input columns that were not output by the
set operation. This commit fixes the problem by removing those
orderings from consideration.

Fixes #73084

Release note (bug fix): Fixed an internal error that could occur
during planning for some set operations (e.g., UNION, INTERSECT or
EXCEPT) when at least one side of the set operation was ordered on
a column that was not output by the set operation. This bug was
first introduced in 21.2.0 and does not exist in prior versions.

73132: bazel: upgrade to `rules_nodejs` 4.4.6, `nodejs` 16.13.0 r=rail a=rickystewart

This comes w/ support for Applie Silicon Macs. Closes #72829.

Release note: None

73136: dev: add `go` command to `dev` r=rail a=rickystewart

Just allows you to run `go` without actually having it installed. Maybe
no one will use this but me, I don't know :)

Release Note: None

73140: CODEOWNERS: move rttanalysis to sql-schema r=postamar a=rafiss

I believe this is more accurate.

Release note: None

Co-authored-by: richardjcai <[email protected]>
Co-authored-by: Azhng <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Marius Posta <[email protected]>
Co-authored-by: Rebecca Taft <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
8 people committed Nov 24, 2021
9 parents 47ce2f0 + 203c7f4 + f35c4a8 + 1cd2a34 + d49bdc1 + bd6ec0c + be76d4c + 8fcd494 + fd5d300 commit 8fa3a38
Show file tree
Hide file tree
Showing 70 changed files with 1,257 additions and 908 deletions.
2 changes: 1 addition & 1 deletion .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@
/pkg/acceptance/ @cockroachdb/sql-experience
/pkg/base/ @cockroachdb/server-prs
/pkg/bench/ @cockroachdb/sql-queries-noreview
/pkg/bench/rttanalysis @cockroachdb/sql-experience
/pkg/bench/rttanalysis @cockroachdb/sql-schema
/pkg/blobs/ @cockroachdb/bulk-prs
/pkg/build/ @cockroachdb/dev-inf
/pkg/ccl/baseccl/ @cockroachdb/cli-prs
Expand Down
6 changes: 3 additions & 3 deletions WORKSPACE
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,9 @@ http_archive(
# Like the above, but for nodeJS.
http_archive(
name = "build_bazel_rules_nodejs",
sha256 = "b32a4713b45095e9e1921a7fcb1adf584bc05959f3336e7351bcf77f015a2d7c",
sha256 = "cfc289523cf1594598215901154a6c2515e8bf3671fd708264a6f6aefe02bf39",
urls = [
"https://storage.googleapis.com/public-bazel-artifacts/bazel/rules_nodejs-4.1.0.tar.gz",
"https://storage.googleapis.com/public-bazel-artifacts/bazel/rules_nodejs-4.4.6.tar.gz",
],
)

Expand Down Expand Up @@ -165,7 +165,7 @@ node_repositories(
node_urls = [
"https://storage.googleapis.com/public-bazel-artifacts/js/node/v{version}/{filename}",
],
node_version = "14.17.5",
node_version = "16.13.0",
package_json = ["//pkg/ui:package.json"],
yarn_urls = [
"https://storage.googleapis.com/public-bazel-artifacts/js/yarn/v{version}/{filename}",
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/streamingccl/streamingutils/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ go_library(
"//pkg/jobs/jobspb",
"//pkg/kv",
"//pkg/sql/sem/tree",
"//pkg/sql/sessiondata",
"//pkg/util/hlc",
"//pkg/util/protoutil",
"//pkg/util/timeutil",
Expand Down
11 changes: 7 additions & 4 deletions pkg/ccl/streamingccl/streamingutils/utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand All @@ -29,8 +30,9 @@ func doCompleteStreamIngestion(
) error {
// Get the job payload for job_id.
const jobsQuery = `SELECT progress FROM system.jobs WHERE id=$1 FOR UPDATE`
row, err := evalCtx.InternalExecutor.QueryRow(evalCtx.Context,
"get-stream-ingestion-job-metadata", txn, jobsQuery, jobID)
row, err := evalCtx.Planner.QueryRowEx(evalCtx.Context,
"get-stream-ingestion-job-metadata",
txn, sessiondata.NodeUserSessionDataOverride, jobsQuery, jobID)
if err != nil {
return err
}
Expand Down Expand Up @@ -82,7 +84,8 @@ func doCompleteStreamIngestion(
return err
}
updateJobQuery := `UPDATE system.jobs SET progress=$1 WHERE id=$2`
_, err = evalCtx.InternalExecutor.QueryRow(evalCtx.Context,
"set-stream-ingestion-job-metadata", txn, updateJobQuery, progressBytes, jobID)
_, err = evalCtx.Planner.QueryRowEx(evalCtx.Context,
"set-stream-ingestion-job-metadata", txn,
sessiondata.NodeUserSessionDataOverride, updateJobQuery, progressBytes, jobID)
return err
}
1 change: 1 addition & 0 deletions pkg/cmd/dev/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"dev.go",
"doctor.go",
"generate.go",
"go.go",
"lint.go",
"logic.go",
"main.go",
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/dev/dev.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ Dev is the general-purpose dev tool for working on cockroachdb/cockroach. With d
makeBuilderCmd(ret.builder),
makeDoctorCmd(ret.doctor),
makeGenerateCmd(ret.generate),
makeGoCmd(ret.gocmd),
makeTestLogicCmd(ret.testlogic),
makeLintCmd(ret.lint),
makeTestCmd(ret.test),
Expand Down
31 changes: 31 additions & 0 deletions pkg/cmd/dev/go.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2020 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package main

import "github.com/spf13/cobra"

func makeGoCmd(runE func(cmd *cobra.Command, args []string) error) *cobra.Command {
return &cobra.Command{
Use: "go <arguments>",
Short: "Run `go` with the given arguments",
Long: "Run `go` with the given arguments",
Example: "dev go mod tidy",
Args: cobra.MinimumNArgs(0),
RunE: runE,
}
}

func (d *dev) gocmd(cmd *cobra.Command, commandLine []string) error {
ctx := cmd.Context()
args := []string{"run", "@go_sdk//:bin/go", "--ui_event_filters=-DEBUG,-info,-stdout,-stderr", "--noshow_progress", "--"}
args = append(args, commandLine...)
return d.exec.CommandContextInheritingStdStreams(ctx, "bazel", args...)
}
2 changes: 2 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -785,6 +785,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
ie.SetSessionData(sessionData)
return &ie
}

distSQLServer.ServerConfig.SessionBoundInternalExecutorFactory = ieFactory
jobRegistry.SetSessionBoundInternalExecutorFactory(ieFactory)
execCfg.IndexBackfiller = sql.NewIndexBackfiller(execCfg, ieFactory)
Expand All @@ -795,6 +796,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
sql.ValidateForwardIndexes,
sql.ValidateInvertedIndexes,
sql.NewFakeSessionData)
execCfg.InternalExecutorFactory = ieFactory

distSQLServer.ServerConfig.ProtectedTimestampProvider = execCfg.ProtectedTimestampProvider

Expand Down
7 changes: 4 additions & 3 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -809,7 +809,7 @@ func (n *alterTableNode) startExec(params runParams) error {
"constraint %q in the middle of being added, try again later", t.Constraint)
}
if err := validateCheckInTxn(
params.ctx, &params.p.semaCtx, params.EvalContext(), n.tableDesc, params.EvalContext().Txn, ck.Expr,
params.ctx, &params.p.semaCtx, params.ExecCfg().InternalExecutor, params.SessionData(), n.tableDesc, params.EvalContext().Txn, ck.Expr,
); err != nil {
return err
}
Expand All @@ -831,7 +831,8 @@ func (n *alterTableNode) startExec(params runParams) error {
"constraint %q in the middle of being added, try again later", t.Constraint)
}
if err := validateFkInTxn(
params.ctx, params.p.LeaseMgr(), params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name,
params.ctx, params.p.LeaseMgr(), params.ExecCfg().InternalExecutor,
n.tableDesc, params.EvalContext().Txn, name, params.EvalContext().Codec,
); err != nil {
return err
}
Expand All @@ -854,7 +855,7 @@ func (n *alterTableNode) startExec(params runParams) error {
"constraint %q in the middle of being added, try again later", t.Constraint)
}
if err := validateUniqueWithoutIndexConstraintInTxn(
params.ctx, params.EvalContext(), n.tableDesc, params.EvalContext().Txn, name,
params.ctx, params.ExecCfg().InternalExecutor, n.tableDesc, params.EvalContext().Txn, name,
); err != nil {
return err
}
Expand Down
29 changes: 14 additions & 15 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,7 +167,7 @@ func (sc *SchemaChanger) makeFixedTimestampInternalExecRunner(
// We need to re-create the evalCtx since the txn may retry.
ie := createSchemaChangeEvalCtx(
ctx, sc.execCfg, readAsOf, sc.ieFactory, descriptors,
).InternalExecutor.(sqlutil.InternalExecutor)
).SchemaChangeInternalExecutor
return retryable(ctx, txn, ie)
})
}
Expand Down Expand Up @@ -710,21 +710,21 @@ func (sc *SchemaChanger) validateConstraints(
defer func() { collection.ReleaseAll(ctx) }()
if c.IsCheck() {
if err := validateCheckInTxn(
ctx, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr,
ctx, &semaCtx, evalCtx.SchemaChangeInternalExecutor, evalCtx.SessionData(), desc, txn, c.Check().Expr,
); err != nil {
return err
}
} else if c.IsForeignKey() {
if err := validateFkInTxn(ctx, sc.leaseMgr, &evalCtx.EvalContext, desc, txn, c.GetName()); err != nil {
if err := validateFkInTxn(ctx, sc.leaseMgr, evalCtx.SchemaChangeInternalExecutor, desc, txn, c.GetName(), evalCtx.Codec); err != nil {
return err
}
} else if c.IsUniqueWithoutIndex() {
if err := validateUniqueWithoutIndexConstraintInTxn(ctx, &evalCtx.EvalContext, desc, txn, c.GetName()); err != nil {
if err := validateUniqueWithoutIndexConstraintInTxn(ctx, evalCtx.SchemaChangeInternalExecutor, desc, txn, c.GetName()); err != nil {
return err
}
} else if c.IsNotNull() {
if err := validateCheckInTxn(
ctx, &semaCtx, &evalCtx.EvalContext, desc, txn, c.Check().Expr,
ctx, &semaCtx, evalCtx.SchemaChangeInternalExecutor, evalCtx.SessionData(), desc, txn, c.Check().Expr,
); err != nil {
// TODO (lucy): This should distinguish between constraint
// validation errors and other types of unexpected errors, and
Expand Down Expand Up @@ -2055,7 +2055,7 @@ func runSchemaChangesInTxn(
check := &c.ConstraintToUpdateDesc().Check
if check.Validity == descpb.ConstraintValidity_Validating {
if err := validateCheckInTxn(
ctx, &planner.semaCtx, planner.EvalContext(), tableDesc, planner.txn, check.Expr,
ctx, &planner.semaCtx, planner.ExecCfg().InternalExecutor, planner.SessionData(), tableDesc, planner.txn, check.Expr,
); err != nil {
return err
}
Expand All @@ -2079,7 +2079,7 @@ func runSchemaChangesInTxn(
uwi := &c.ConstraintToUpdateDesc().UniqueWithoutIndexConstraint
if uwi.Validity == descpb.ConstraintValidity_Validating {
if err := validateUniqueWithoutIndexConstraintInTxn(
ctx, planner.EvalContext(), tableDesc, planner.txn, c.GetName(),
ctx, planner.ExecCfg().InternalExecutor, tableDesc, planner.txn, c.GetName(),
); err != nil {
return err
}
Expand Down Expand Up @@ -2152,18 +2152,18 @@ func runSchemaChangesInTxn(
func validateCheckInTxn(
ctx context.Context,
semaCtx *tree.SemaContext,
evalCtx *tree.EvalContext,
ie *InternalExecutor,
sessionData *sessiondata.SessionData,
tableDesc *tabledesc.Mutable,
txn *kv.Txn,
checkExpr string,
) error {
ie := evalCtx.InternalExecutor.(*InternalExecutor)
var syntheticDescs []catalog.Descriptor
if tableDesc.Version > tableDesc.ClusterVersion.Version {
syntheticDescs = append(syntheticDescs, tableDesc)
}
return ie.WithSyntheticDescriptors(syntheticDescs, func() error {
return validateCheckExpr(ctx, semaCtx, evalCtx.SessionData(), checkExpr, tableDesc, ie, txn)
return validateCheckExpr(ctx, semaCtx, sessionData, checkExpr, tableDesc, ie, txn)
})
}

Expand All @@ -2182,12 +2182,12 @@ func validateCheckInTxn(
func validateFkInTxn(
ctx context.Context,
leaseMgr *lease.Manager,
evalCtx *tree.EvalContext,
ie *InternalExecutor,
tableDesc *tabledesc.Mutable,
txn *kv.Txn,
fkName string,
codec keys.SQLCodec,
) error {
ie := evalCtx.InternalExecutor.(*InternalExecutor)
var syntheticDescs []catalog.Descriptor
if tableDesc.Version > tableDesc.ClusterVersion.Version {
syntheticDescs = append(syntheticDescs, tableDesc)
Expand All @@ -2206,7 +2206,7 @@ func validateFkInTxn(
}

return ie.WithSyntheticDescriptors(syntheticDescs, func() error {
return validateForeignKey(ctx, tableDesc, fk, ie, txn, evalCtx.Codec)
return validateForeignKey(ctx, tableDesc, fk, ie, txn, codec)
})
}

Expand All @@ -2224,12 +2224,11 @@ func validateFkInTxn(
// reuse an existing kv.Txn safely.
func validateUniqueWithoutIndexConstraintInTxn(
ctx context.Context,
evalCtx *tree.EvalContext,
ie *InternalExecutor,
tableDesc *tabledesc.Mutable,
txn *kv.Txn,
constraintName string,
) error {
ie := evalCtx.InternalExecutor.(*InternalExecutor)
var syntheticDescs []catalog.Descriptor
if tableDesc.Version > tableDesc.ClusterVersion.Version {
syntheticDescs = append(syntheticDescs, tableDesc)
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,11 @@ type PostDeserializationTableDescriptorChanges struct {

// UpgradedPrivileges indicates that the PrivilegeDescriptor version was upgraded.
UpgradedPrivileges bool

// RemovedDefaultExprFromComputedColumn indicates that the table had at least
// one computed column which also had a DEFAULT expression, which therefore
// had to be removed. See issue #72881 for details.
RemovedDefaultExprFromComputedColumn bool
}

// DescriptorType returns the type of this descriptor.
Expand Down
48 changes: 48 additions & 0 deletions pkg/sql/catalog/tabledesc/structured_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -821,6 +821,54 @@ func TestDefaultExprNil(t *testing.T) {
})
}

// TestRemoveDefaultExprFromComputedColumn tests that default expressions are
// correctly removed from descriptors of computed columns as part of the
// RunPostDeserializationChanges suite.
func TestRemoveDefaultExprFromComputedColumn(t *testing.T) {
s, sqlDB, kvDB := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(context.Background())
tdb := sqlutils.MakeSQLRunner(sqlDB)

const expectedErrRE = `.*: computed column \"b\" cannot also have a DEFAULT expression`
// Create a table with a computed column.
tdb.Exec(t, `CREATE DATABASE t`)
tdb.Exec(t, `CREATE TABLE t.tbl (a INT PRIMARY KEY, b INT AS (1) STORED)`)

// Get the descriptor for the table.
tbl := catalogkv.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, "t", "tbl")

// Setting a default value on the computed column should fail.
tdb.ExpectErr(t, expectedErrRE, `ALTER TABLE t.tbl ALTER COLUMN b SET DEFAULT 2`)

// Copy the descriptor proto for the table and modify it by setting a default
// expression.
var desc *descpb.TableDescriptor
{
desc = NewBuilder(tbl.TableDesc()).BuildImmutableTable().TableDesc()
defaultExpr := "2"
desc.Columns[1].DefaultExpr = &defaultExpr
}

// This modified table descriptor should fail validation.
{
broken := NewBuilder(desc).BuildImmutableTable()
require.Error(t, catalog.ValidateSelf(broken))
}

// This modified table descriptor should be fixed by removing the default
// expression.
{
b := NewBuilder(desc)
require.NoError(t, b.RunPostDeserializationChanges(context.Background(), nil /* dg */))
fixed := b.BuildImmutableTable()
require.NoError(t, catalog.ValidateSelf(fixed))
changes, err := GetPostDeserializationChanges(fixed)
require.NoError(t, err)
require.True(t, changes.RemovedDefaultExprFromComputedColumn)
require.False(t, fixed.PublicColumns()[1].HasDefault())
}
}

func TestLogicalColumnID(t *testing.T) {
tests := []struct {
desc descpb.TableDescriptor
Expand Down
25 changes: 25 additions & 0 deletions pkg/sql/catalog/tabledesc/table_desc_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -204,6 +204,7 @@ func maybeFillInDescriptor(
}
}
changes.UpgradedNamespaceName = maybeUpgradeNamespaceName(desc)
changes.RemovedDefaultExprFromComputedColumn = maybeRemoveDefaultExprFromComputedColumns(desc)

parentSchemaID := desc.GetUnexposedParentSchemaID()
if parentSchemaID == descpb.InvalidID {
Expand All @@ -227,6 +228,30 @@ func maybeFillInDescriptor(
return changes, nil
}

// maybeRemoveDefaultExprFromComputedColumns removes DEFAULT expressions on
// computed columns. Although we now have a descriptor validation check to
// prevent this, this hasn't always been the case, so it's theoretically
// possible to encounter table descriptors which would fail this validation
// check. See issue #72881 for details.
func maybeRemoveDefaultExprFromComputedColumns(desc *descpb.TableDescriptor) (hasChanged bool) {
doCol := func(col *descpb.ColumnDescriptor) {
if col.IsComputed() && col.HasDefault() {
col.DefaultExpr = nil
hasChanged = true
}
}

for i := range desc.Columns {
doCol(&desc.Columns[i])
}
for _, m := range desc.Mutations {
if col := m.GetColumn(); col != nil && m.Direction != descpb.DescriptorMutation_DROP {
doCol(col)
}
}
return hasChanged
}

// maybeUpgradeForeignKeyRepresentation destructively modifies the input table
// descriptor by replacing all old-style foreign key references (the ForeignKey
// and ReferencedBy fields on IndexDescriptor) with new-style foreign key
Expand Down
Loading

0 comments on commit 8fa3a38

Please sign in to comment.