From 7d148f6894e9ef49f2c50a3037dd10ceda410311 Mon Sep 17 00:00:00 2001 From: Paul Bardea Date: Fri, 26 Feb 2021 17:18:33 -0500 Subject: [PATCH 1/9] importccl: avoid random_number collisions between KV batches An import may parallelize the work to convert SQL rows into KVs. During this phase, default expressions are evaluated. Previously, IMPORT's implementations of random number generates that are evaluated in default expressions assumed that all of the given row IDs were contiguous. This is not the case since 1 row converter may be responsible for converting several non-contiguous batches of rows. This resulted in random_values colliding between different batches of KV space. This commit fixes this bug by feeding in the current position and resetting the random source backing these methods. This ensures that import treats a new contiguous batch of rows separately. Release justification: bug fix Release note (bug fix): Fix a bug where random numbers generated as default expressions during IMPORT would collide a few hundred rows apart from each-other. --- pkg/ccl/importccl/import_stmt_test.go | 54 +++++++++++-- pkg/sql/row/expr_walker.go | 107 +++++++++++++++++++------- 2 files changed, 128 insertions(+), 33 deletions(-) diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go index f19add2b194e..d112e3b7425d 100644 --- a/pkg/ccl/importccl/import_stmt_test.go +++ b/pkg/ccl/importccl/import_stmt_test.go @@ -3968,7 +3968,7 @@ func TestImportDefault(t *testing.T) { } }) - t.Run("random-related", func(t *testing.T) { + t.Run("random-functions", func(t *testing.T) { testCases := []struct { name string create string @@ -3978,19 +3978,19 @@ func TestImportDefault(t *testing.T) { }{ { name: "random-multiple", - create: "a INT, b FLOAT DEFAULT random(), c STRING, d FLOAT DEFAULT random()", + create: "a INT, b FLOAT PRIMARY KEY DEFAULT random(), c STRING, d FLOAT DEFAULT random()", targetCols: []string{"a", "c"}, randomCols: []string{selectNotNull("b"), selectNotNull("d")}, }, { name: "gen_random_uuid", - create: "a INT, b STRING, c UUID DEFAULT gen_random_uuid()", + create: "a INT, b STRING, c UUID PRIMARY KEY DEFAULT gen_random_uuid(), d UUID DEFAULT gen_random_uuid()", targetCols: []string{"a", "b"}, - randomCols: []string{selectNotNull("c")}, + randomCols: []string{selectNotNull("c"), selectNotNull("d")}, }, { name: "mixed_random_uuid", - create: "a INT, b STRING, c UUID DEFAULT gen_random_uuid(), d FLOAT DEFAULT random()", + create: "a INT, b STRING, c UUID PRIMARY KEY DEFAULT gen_random_uuid(), d FLOAT DEFAULT random()", targetCols: []string{"a", "b"}, randomCols: []string{selectNotNull("c")}, }, @@ -4033,6 +4033,50 @@ func TestImportDefault(t *testing.T) { }) } +// This is a regression test for #61203. We test that the random() keys are +// unique on a larger data set. This would previously fail with a primary key +// collision error since we would generate duplicate UUIDs. +// +// Note: that although there is no guarantee that UUIDs do not collide, the +// probability of such a collision is vanishingly low. +func TestUniqueUUID(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // This test is slow under race since it explicitly tried to import a large + // amount of data. + skip.UnderRace(t, "slow under race") + + const ( + nodes = 3 + dataDir = "userfile://defaultdb.my_files/export" + dataFiles = dataDir + "/*" + ) + ctx := context.Background() + args := base.TestServerArgs{} + tc := testcluster.StartTestCluster(t, nodes, base.TestClusterArgs{ServerArgs: args}) + defer tc.Stopper().Stop(ctx) + connDB := tc.Conns[0] + sqlDB := sqlutils.MakeSQLRunner(connDB) + + dataSize := parallelImporterReaderBatchSize * 100 + + sqlDB.Exec(t, fmt.Sprintf(`CREATE TABLE data AS SELECT * FROM generate_series(1, %d);`, dataSize)) + sqlDB.Exec(t, `EXPORT INTO CSV $1 FROM TABLE data;`, dataDir) + + // Ensure that UUIDs do not collide when importing 20000 rows. + sqlDB.Exec(t, `CREATE TABLE r1 (a UUID PRIMARY KEY DEFAULT gen_random_uuid(), b INT);`) + sqlDB.Exec(t, `IMPORT INTO r1 (b) CSV DATA ($1);`, dataFiles) + + // Ensure that UUIDs do not collide when importing into a table with several UUID calls. + sqlDB.Exec(t, `CREATE TABLE r2 (a UUID PRIMARY KEY DEFAULT gen_random_uuid(), b INT, c UUID DEFAULT gen_random_uuid());`) + sqlDB.Exec(t, `IMPORT INTO r2 (b) CSV DATA ($1);`, dataFiles) + + // Ensure that random keys do not collide. + sqlDB.Exec(t, `CREATE TABLE r3 (a FLOAT PRIMARY KEY DEFAULT random(), b INT);`) + sqlDB.Exec(t, `IMPORT INTO r3 (b) CSV DATA ($1);`, dataFiles) +} + func TestImportDefaultNextVal(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/sql/row/expr_walker.go b/pkg/sql/row/expr_walker.go index 26c7102bc996..ffb82249690e 100644 --- a/pkg/sql/row/expr_walker.go +++ b/pkg/sql/row/expr_walker.go @@ -45,48 +45,93 @@ const chunkSizeIncrementRate = 10 const initialChunkSize = 10 const maxChunkSize = 100000 +// importRandPosition uniquely identifies an instance to a call to a random +// function during an import. +type importRandPosition int64 + +func (pos importRandPosition) distance(o importRandPosition) int64 { + diff := int64(pos) - int64(o) + if diff < 0 { + return -diff + } + return diff +} + +// getPosForRandImport gives the importRandPosition for the first instance of a +// call to a random function when generating a given row from a given source. +// numInstances refers to the number of random function invocations per row. +func getPosForRandImport(rowID int64, sourceID int32, numInstances int) importRandPosition { + // We expect r.pos to increment by numInstances for each row. + // Therefore, assuming that rowID increments by 1 for every row, + // we will initialize the position as rowID * numInstances + sourceID << rowIDBits. + rowIDWithMultiplier := int64(numInstances) * rowID + pos := (int64(sourceID) << rowIDBits) ^ rowIDWithMultiplier + return importRandPosition(pos) +} + +// randomSource is only exposed through an interface to ensure that caller's +// don't access underlying field. +type randomSource interface { + // Float64 returns, as a float64, a pseudo-random number in [0.0,1.0). + Float64(c *CellInfoAnnotation) float64 + // Int63 returns a non-negative pseudo-random 63-bit integer as an int64. + Int63(c *CellInfoAnnotation) int64 +} + +var _ randomSource = (*importRand)(nil) + type importRand struct { *rand.Rand - pos int64 + pos importRandPosition } -func newImportRand(pos int64) *importRand { +func (r *importRand) reseed(pos importRandPosition) { adjPos := (pos / reseedRandEveryN) * reseedRandEveryN - rnd := rand.New(rand.NewSource(adjPos)) + rnd := rand.New(rand.NewSource(int64(adjPos))) for i := int(pos % reseedRandEveryN); i > 0; i-- { _ = rnd.Float64() } - return &importRand{rnd, pos} -} -func (r *importRand) advancePos() { - r.pos++ + r.Rand = rnd + r.pos = pos +} + +func (r *importRand) maybeReseed(c *CellInfoAnnotation) { + // newRowPos is the position of the first random function invocation of the + // row we're currently processing. If this is more than c.randInstancePerRow + // away, that means that we've skipped a batch of rows. importRand assumes + // that it operates on a contiguous set of rows when it increments its + // position, so if we skip some rows we need to reseed. + // We may skip rows because a single row converter may be responsible for + // converting several non-contiguous batches of KVs. + newRowPos := getPosForRandImport(c.rowID, c.sourceID, c.randInstancePerRow) + rowsSkipped := newRowPos.distance(r.pos) > int64(c.randInstancePerRow) + if rowsSkipped { + // Reseed at the new position, since our internally tracked r.pos is now out + // of sync. + r.reseed(newRowPos) + } if r.pos%reseedRandEveryN == 0 { - // Time to reseed. - r.Rand = rand.New(rand.NewSource(r.pos)) + r.reseed(r.pos) } } -func (r *importRand) Float64() float64 { +// Float64 implements the randomSource interface. +func (r *importRand) Float64(c *CellInfoAnnotation) float64 { + r.maybeReseed(c) randNum := r.Rand.Float64() - r.advancePos() + r.pos++ return randNum } -func (r *importRand) Int63() int64 { +// Int63 implements the randomSource interface. +func (r *importRand) Int63(c *CellInfoAnnotation) int64 { + r.maybeReseed(c) randNum := r.Rand.Int63() - r.advancePos() + r.pos++ return randNum } -func getSeedForImportRand(rowID int64, sourceID int32, numInstances int) int64 { - // We expect r.pos to increment by numInstances for each row. - // Therefore, assuming that rowID increments by 1 for every row, - // we will initialize the position as rowID * numInstances + sourceID << rowIDBits. - rowIDWithMultiplier := int64(numInstances) * rowID - return (int64(sourceID) << rowIDBits) ^ rowIDWithMultiplier -} - // For some functions (specifically the volatile ones), we do // not want to use the provided builtin. Instead, we opt for // our own function definition, which produces deterministic results. @@ -135,7 +180,8 @@ type CellInfoAnnotation struct { uniqueRowIDTotal int // Annotations for rand() and gen_random_uuid(). - randSource *importRand + // randSource should not be used directly, but through getImportRand() instead. + randSource randomSource randInstancePerRow int // Annotations for next_val(). @@ -154,6 +200,13 @@ func (c *CellInfoAnnotation) reset(sourceID int32, rowID int64) { c.uniqueRowIDInstance = 0 } +func makeImportRand(c *CellInfoAnnotation) randomSource { + pos := getPosForRandImport(c.rowID, c.sourceID, c.randInstancePerRow) + randSource := &importRand{} + randSource.reseed(pos) + return randSource +} + // We don't want to call unique_rowid() for columns with such default expressions // because it is not idempotent and has unfortunate overlapping of output // spans since it puts the uniqueness-ensuring per-generator part (nodeID) @@ -202,19 +255,17 @@ func importUniqueRowID(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, func importRandom(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { c := getCellInfoAnnotation(evalCtx.Annotations) if c.randSource == nil { - c.randSource = newImportRand(getSeedForImportRand( - c.rowID, c.sourceID, c.randInstancePerRow)) + c.randSource = makeImportRand(c) } - return tree.NewDFloat(tree.DFloat(c.randSource.Float64())), nil + return tree.NewDFloat(tree.DFloat(c.randSource.Float64(c))), nil } func importGenUUID(evalCtx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { c := getCellInfoAnnotation(evalCtx.Annotations) if c.randSource == nil { - c.randSource = newImportRand(getSeedForImportRand( - c.rowID, c.sourceID, c.randInstancePerRow)) + c.randSource = makeImportRand(c) } - gen := c.randSource.Int63() + gen := c.randSource.Int63(c) id := uuid.MakeV4() id.DeterministicV4(uint64(gen), uint64(1<<63)) return tree.NewDUuid(tree.DUuid{UUID: id}), nil From d5f73c6a05c99f7c6b40b80dea80e8ed8ab15cac Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sat, 27 Feb 2021 00:00:04 +0100 Subject: [PATCH 2/9] grpcutil: rename IsAuthError and handle PermissionDenied `IsAuthenticationError` is often used to return RPC errors immediately instead of retrying them. Previously, this only considered `codes.Unauthenticated` but not `codes.PermissionDenied`. This could cause various operations to run indefinitely due to internal RPC retries, e.g. after a node has been decommissioned and lost access to the cluster. Since permission errors should also return immediately, this patch changes the function to handle `codes.PermissionDenied` as well. However, since a missing permission is an authorization failure rather than an authentication failure, the function was renamed to `IsAuthError` such that it can cover both authn and authz failures. Release justification: low risk, high benefit changes to existing functionality Release note (bug fix): Fix operation hangs when a node loses access to cluster RPC (e.g. after it has been decommissioned), and immediately return an error instead. --- pkg/ccl/kvccl/kvtenantccl/connector.go | 4 ++-- pkg/ccl/kvccl/kvtenantccl/connector_test.go | 2 +- pkg/kv/kvclient/kvcoord/dist_sender.go | 4 ++-- pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go | 4 ++-- pkg/kv/kvclient/rangecache/range_cache.go | 4 ++-- pkg/server/settingswatcher/settings_watcher.go | 2 +- pkg/util/grpcutil/grpc_util.go | 11 +++++++---- 7 files changed, 17 insertions(+), 14 deletions(-) diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index 6721c0ca47cd..db6876362fb7 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -329,8 +329,8 @@ func (c *Connector) RangeLookup( }) if err != nil { log.Warningf(ctx, "error issuing RangeLookup RPC: %v", err) - if grpcutil.IsAuthenticationError(err) { - // Authentication error. Propagate. + if grpcutil.IsAuthError(err) { + // Authentication or authorization error. Propagate. return nil, nil, err } // Soft RPC error. Drop client and retry. diff --git a/pkg/ccl/kvccl/kvtenantccl/connector_test.go b/pkg/ccl/kvccl/kvtenantccl/connector_test.go index 8cda06c9306c..3e845e3f81fe 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector_test.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector_test.go @@ -330,7 +330,7 @@ func TestConnectorRangeLookup(t *testing.T) { desc, err := c.FirstRange() require.Nil(t, desc) require.Regexp(t, "does not have access to FirstRange", err) - require.True(t, grpcutil.IsAuthenticationError(err)) + require.True(t, grpcutil.IsAuthError(err)) } // TestConnectorRetriesUnreachable tests that Connector iterates over each of diff --git a/pkg/kv/kvclient/kvcoord/dist_sender.go b/pkg/kv/kvclient/kvcoord/dist_sender.go index 9a2c6ef6f609..7748a352b345 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender.go @@ -1888,8 +1888,8 @@ func (ds *DistSender) sendToReplicas( ds.maybeIncrementErrCounters(br, err) if err != nil { - if grpcutil.IsAuthenticationError(err) { - // Authentication error. Propagate. + if grpcutil.IsAuthError(err) { + // Authentication or authorization error. Propagate. if ambiguousError != nil { return nil, roachpb.NewAmbiguousResultErrorf("error=%s [propagate]", ambiguousError) } diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go index db8acb309da8..0b0495966ed9 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed.go @@ -260,8 +260,8 @@ func (ds *DistSender) singleRangeFeed( stream, err := client.RangeFeed(clientCtx, &args) if err != nil { log.VErrEventf(ctx, 2, "RPC error: %s", err) - if grpcutil.IsAuthenticationError(err) { - // Authentication error. Propagate. + if grpcutil.IsAuthError(err) { + // Authentication or authorization error. Propagate. return args.Timestamp, err } continue diff --git a/pkg/kv/kvclient/rangecache/range_cache.go b/pkg/kv/kvclient/rangecache/range_cache.go index 30272a00edb8..bf93abd2b72b 100644 --- a/pkg/kv/kvclient/rangecache/range_cache.go +++ b/pkg/kv/kvclient/rangecache/range_cache.go @@ -1324,6 +1324,6 @@ func (e *CacheEntry) evictLeaseholder( // IsRangeLookupErrorRetryable returns whether the provided range lookup error // can be retried or whether it should be propagated immediately. func IsRangeLookupErrorRetryable(err error) bool { - // For now, all errors are retryable except authentication errors. - return !grpcutil.IsAuthenticationError(err) + // For now, all errors are retryable except authentication/authorization. + return !grpcutil.IsAuthError(err) } diff --git a/pkg/server/settingswatcher/settings_watcher.go b/pkg/server/settingswatcher/settings_watcher.go index a5ca60de0996..ae0d67c6d581 100644 --- a/pkg/server/settingswatcher/settings_watcher.go +++ b/pkg/server/settingswatcher/settings_watcher.go @@ -106,7 +106,7 @@ func (s *SettingsWatcher) Start(ctx context.Context) error { ) (shouldFail bool) { // TODO(ajwerner): Consider if there are other errors which we want to // treat as permanent. - if grpcutil.IsAuthenticationError(err) || + if grpcutil.IsAuthError(err) || // This is a hack around the fact that we do not get properly structured // errors out of gRPC. See #56208. strings.Contains(err.Error(), "rpc error: code = Unauthenticated") { diff --git a/pkg/util/grpcutil/grpc_util.go b/pkg/util/grpcutil/grpc_util.go index 231f0a4b577c..eda8ef94c370 100644 --- a/pkg/util/grpcutil/grpc_util.go +++ b/pkg/util/grpcutil/grpc_util.go @@ -91,11 +91,14 @@ func IsClosedConnection(err error) bool { return netutil.IsClosedConnection(err) } -// IsAuthenticationError returns true if err's Cause is an error produced by -// gRPC due to invalid authentication credentials for the operation. -func IsAuthenticationError(err error) bool { +// IsAuthError returns true if err's Cause is an error produced by +// gRPC due to an authentication or authorization error for the operation. +func IsAuthError(err error) bool { if s, ok := status.FromError(errors.UnwrapAll(err)); ok { - return s.Code() == codes.Unauthenticated + switch s.Code() { + case codes.Unauthenticated, codes.PermissionDenied: + return true + } } return false } From 247f251a25409f62dfcd429b31013bd219f06054 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Mon, 1 Mar 2021 13:37:50 +0100 Subject: [PATCH 3/9] server: move TestDecommissionNodeStatus into server_test.go The test `TestDecommissionNodeStatus` was initially placed in a separate file `decommission_test.go`, even though it belonged in `server_test.go`. This was done to avoid import cycles between the `testcluster` package and the `server` package, as the `decommission_test.go` file could use the `server_test` package instead. This commit creates the cluster with `serverutils.StartNewTestCluster()` instead of `testcluster.StartTestCluster()`, which avoids the import cycle and allows it to be moved into `server_test.go`. This required exposing `Server.Decommission()` via `serverutils.TestServerInterface` as well. Release justification: non-production code changes Release note: None --- pkg/server/BUILD.bazel | 1 - pkg/server/decommission_test.go | 57 ------------------- pkg/server/server_test.go | 33 +++++++++++ pkg/testutils/serverutils/BUILD.bazel | 1 + pkg/testutils/serverutils/test_server_shim.go | 4 ++ 5 files changed, 38 insertions(+), 58 deletions(-) delete mode 100644 pkg/server/decommission_test.go diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 695c45507ce3..0616a5c9bf47 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -245,7 +245,6 @@ go_test( "auto_tls_init_test.go", "config_test.go", "connectivity_test.go", - "decommission_test.go", "drain_test.go", "graphite_test.go", "idle_monitor_test.go", diff --git a/pkg/server/decommission_test.go b/pkg/server/decommission_test.go deleted file mode 100644 index abf8cdb7dcee..000000000000 --- a/pkg/server/decommission_test.go +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright 2016 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 server_test - -import ( - "context" - "testing" - - "github.com/cockroachdb/cockroach/pkg/base" - "github.com/cockroachdb/cockroach/pkg/keys" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" - "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/stretchr/testify/require" -) - -// This test should really be in server_test.go but can't because it uses the -// server package, which is also imported by testutils/testcluster, causing -// an import cycle. -func TestDecommissionNodeStatus(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - ctx := context.Background() - tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{ - ReplicationMode: base.ReplicationManual, // saves time - }) - defer tc.Stopper().Stop(ctx) - decomNodeID := tc.Server(2).NodeID() - - // Make sure node status entries have been created. - for _, srv := range tc.Servers { - entry, err := srv.DB().Get(ctx, keys.NodeStatusKey(srv.NodeID())) - require.NoError(t, err) - require.NotNil(t, entry.Value, "node status entry not found for node %d", srv.NodeID()) - } - - require.NoError(t, tc.Servers[0].Decommission( - ctx, livenesspb.MembershipStatus_DECOMMISSIONING, []roachpb.NodeID{decomNodeID})) - require.NoError(t, tc.Servers[0].Decommission( - ctx, livenesspb.MembershipStatus_DECOMMISSIONED, []roachpb.NodeID{decomNodeID})) - - // The node status entry should now have been cleaned up. - entry, err := tc.Server(0).DB().Get(ctx, keys.NodeStatusKey(decomNodeID)) - require.NoError(t, err) - require.Nil(t, entry.Value, "found stale node status entry for node %d", decomNodeID) -} diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go index 97c0b3be7e1c..b95cc23606fa 100644 --- a/pkg/server/server_test.go +++ b/pkg/server/server_test.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -1130,3 +1131,35 @@ func TestGWRuntimeMarshalProto(t *testing.T) { errors.New("boom"), ) } + +func TestDecommissionNodeStatus(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, // saves time + }) + defer tc.Stopper().Stop(ctx) + decomNodeID := tc.Server(2).NodeID() + + // Make sure node status entries have been created. + for i := 0; i < tc.NumServers(); i++ { + srv := tc.Server(i) + entry, err := srv.DB().Get(ctx, keys.NodeStatusKey(srv.NodeID())) + require.NoError(t, err) + require.NotNil(t, entry.Value, "node status entry not found for node %d", srv.NodeID()) + } + + // Decommission the node. + srv := tc.Server(0) + require.NoError(t, srv.Decommission( + ctx, livenesspb.MembershipStatus_DECOMMISSIONING, []roachpb.NodeID{decomNodeID})) + require.NoError(t, srv.Decommission( + ctx, livenesspb.MembershipStatus_DECOMMISSIONED, []roachpb.NodeID{decomNodeID})) + + // The node status entry should now have been cleaned up. + entry, err := srv.DB().Get(ctx, keys.NodeStatusKey(decomNodeID)) + require.NoError(t, err) + require.Nil(t, entry.Value, "found stale node status entry for node %d", decomNodeID) +} diff --git a/pkg/testutils/serverutils/BUILD.bazel b/pkg/testutils/serverutils/BUILD.bazel index 224bfa9f851c..ba892d2fac39 100644 --- a/pkg/testutils/serverutils/BUILD.bazel +++ b/pkg/testutils/serverutils/BUILD.bazel @@ -12,6 +12,7 @@ go_library( deps = [ "//pkg/base", "//pkg/kv", + "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/rpc", "//pkg/security", diff --git a/pkg/testutils/serverutils/test_server_shim.go b/pkg/testutils/serverutils/test_server_shim.go index 24fd40eba088..911fbe3473c0 100644 --- a/pkg/testutils/serverutils/test_server_shim.go +++ b/pkg/testutils/serverutils/test_server_shim.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" @@ -180,6 +181,9 @@ type TestServerInterface interface { // this server. ClusterSettings() *cluster.Settings + // Decommission idempotently sets the decommissioning flag for specified nodes. + Decommission(ctx context.Context, targetStatus livenesspb.MembershipStatus, nodeIDs []roachpb.NodeID) error + // SplitRange splits the range containing splitKey. SplitRange(splitKey roachpb.Key) (left roachpb.RangeDescriptor, right roachpb.RangeDescriptor, err error) From 6355b060295a223d536a21e2d7b4a14ec5c55395 Mon Sep 17 00:00:00 2001 From: Marcus Gartner Date: Tue, 2 Mar 2021 10:58:37 -0800 Subject: [PATCH 4/9] opt: prune unnecessary columns in uniqueness checks Projects now wrap semi-joins of unique checks which pass-through the columns of the unique constraint. This allows normalization rules to prune unnecessary columns from the expression. Release justification: This is a low-risk change to new functionality, implicitly partitioned unique indexes. Release note (performance improvement): The columns fetched for uniqueness checks of implicitly partitioned unique indexes are now pruned to only include columns necessary for determining uniqueness. --- .../testdata/logic_test/partitioning_implicit | 2 - pkg/sql/opt/exec/execbuilder/testdata/unique | 575 +++---- pkg/sql/opt/norm/testdata/rules/prune_cols | 392 ++--- .../opt/optbuilder/mutation_builder_unique.go | 10 +- .../optbuilder/testdata/unique-checks-insert | 1204 +++++++------- .../optbuilder/testdata/unique-checks-update | 888 +++++----- .../optbuilder/testdata/unique-checks-upsert | 1453 +++++++++-------- 7 files changed, 2383 insertions(+), 2141 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit index 6d77ed7c0843..bb54d159f253 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit +++ b/pkg/ccl/logictestccl/testdata/logic_test/partitioning_implicit @@ -656,7 +656,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 1 │ label: buffer 1 │ └── • constraint-check @@ -674,7 +673,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 1 label: buffer 1 statement ok diff --git a/pkg/sql/opt/exec/execbuilder/testdata/unique b/pkg/sql/opt/exec/execbuilder/testdata/unique index f5f86a128044..13c1ebb85b6f 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/unique +++ b/pkg/sql/opt/exec/execbuilder/testdata/unique @@ -271,7 +271,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 2 │ label: buffer 1 │ └── • constraint-check @@ -288,7 +287,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 2 label: buffer 1 # No need to plan checks for w since it's always null. @@ -324,7 +322,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 2 label: buffer 1 # Use all the unique indexes and constraints as arbiters for DO NOTHING with no @@ -478,7 +475,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 2 │ label: buffer 1 │ ├── • constraint-check @@ -491,7 +487,6 @@ vectorized: true │ │ pred: column2 != b │ │ │ └── • scan buffer -│ estimated row count: 2 │ label: buffer 1 │ └── • constraint-check @@ -617,7 +612,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 2 │ label: buffer 1 │ └── • constraint-check @@ -634,7 +628,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 2 label: buffer 1 # Combine unique checks with foreign keys. There should be two foreign key @@ -673,7 +666,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 2 │ label: buffer 1 │ ├── • constraint-check @@ -752,34 +744,38 @@ vectorized: true │ │ columns: () │ │ │ └── • project -│ │ columns: (column1, column2, column3, column4) +│ │ columns: (column3) │ │ estimated row count: 1 (missing stats) │ │ -│ └── • lookup join (semi) -│ │ columns: ("lookup_join_const_col_@12", column1, column2, column3, column4) -│ │ table: uniq_enum@primary -│ │ equality: (lookup_join_const_col_@12, column3) = (r,i) -│ │ equality cols are key -│ │ pred: column1 != r +│ └── • project +│ │ columns: (column1, column3) +│ │ estimated row count: 1 (missing stats) │ │ -│ └── • cross join (inner) -│ │ columns: ("lookup_join_const_col_@12", column1, column2, column3, column4) -│ │ estimated row count: 6 +│ └── • lookup join (semi) +│ │ columns: ("lookup_join_const_col_@12", column1, column3) +│ │ table: uniq_enum@primary +│ │ equality: (lookup_join_const_col_@12, column3) = (r,i) +│ │ equality cols are key +│ │ pred: column1 != r │ │ -│ ├── • values -│ │ columns: ("lookup_join_const_col_@12") -│ │ size: 1 column, 3 rows -│ │ row 0, expr 0: 'us-east' -│ │ row 1, expr 0: 'us-west' -│ │ row 2, expr 0: 'eu-west' -│ │ -│ └── • project -│ │ columns: (column1, column2, column3, column4) -│ │ estimated row count: 2 +│ └── • cross join (inner) +│ │ columns: ("lookup_join_const_col_@12", column1, column3) +│ │ estimated row count: 6 +│ │ +│ ├── • values +│ │ columns: ("lookup_join_const_col_@12") +│ │ size: 1 column, 3 rows +│ │ row 0, expr 0: 'us-east' +│ │ row 1, expr 0: 'us-west' +│ │ row 2, expr 0: 'eu-west' │ │ -│ └── • scan buffer -│ columns: (column1, column2, column3, column4, check1) -│ label: buffer 1 +│ └── • project +│ │ columns: (column1, column3) +│ │ estimated row count: 2 +│ │ +│ └── • scan buffer +│ columns: (column1, column2, column3, column4, check1) +│ label: buffer 1 │ └── • constraint-check │ @@ -787,34 +783,38 @@ vectorized: true │ columns: () │ └── • project - │ columns: (column1, column2, column3, column4) + │ columns: (column2, column4) │ estimated row count: 1 (missing stats) │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@22", column1, column2, column3, column4) - │ table: uniq_enum@uniq_enum_r_s_j_key - │ equality: (lookup_join_const_col_@22, column2, column4) = (r,s,j) - │ equality cols are key - │ pred: (column1 != r) OR (column3 != i) + └── • project + │ columns: (column1, column2, column3, column4) + │ estimated row count: 1 (missing stats) │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: ("lookup_join_const_col_@22", column1, column2, column3, column4) - │ estimated row count: 6 - │ - ├── • values - │ columns: ("lookup_join_const_col_@22") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + │ table: uniq_enum@uniq_enum_r_s_j_key + │ equality: (lookup_join_const_col_@22, column2, column4) = (r,s,j) + │ equality cols are key + │ pred: (column1 != r) OR (column3 != i) │ - └── • project - │ columns: (column1, column2, column3, column4) - │ estimated row count: 2 + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@22", column1, column2, column3, column4) + │ estimated row count: 6 + │ + ├── • values + │ columns: ("lookup_join_const_col_@22") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' │ - └── • scan buffer - columns: (column1, column2, column3, column4, check1) - label: buffer 1 + └── • project + │ columns: (column1, column2, column3, column4) + │ estimated row count: 2 + │ + └── • scan buffer + columns: (column1, column2, column3, column4, check1) + label: buffer 1 # Test that we use the index when available for the insert checks. This uses # the default value for columns r and j. @@ -867,34 +867,38 @@ vectorized: true │ columns: () │ └── • project - │ columns: (column9, column1, column2, column10) + │ columns: (column2) │ estimated row count: 1 (missing stats) │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@12", column9, column1, column2, column10) - │ table: uniq_enum@primary - │ equality: (lookup_join_const_col_@12, column2) = (r,i) - │ equality cols are key - │ pred: column9 != r + └── • project + │ columns: (column9, column2) + │ estimated row count: 1 (missing stats) │ - └── • cross join (inner) - │ columns: ("lookup_join_const_col_@12", column9, column1, column2, column10) - │ estimated row count: 6 - │ - ├── • values - │ columns: ("lookup_join_const_col_@12") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + └── • lookup join (semi) + │ columns: ("lookup_join_const_col_@12", column9, column2) + │ table: uniq_enum@primary + │ equality: (lookup_join_const_col_@12, column2) = (r,i) + │ equality cols are key + │ pred: column9 != r │ - └── • project - │ columns: (column9, column1, column2, column10) - │ estimated row count: 2 + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@12", column9, column2) + │ estimated row count: 6 + │ + ├── • values + │ columns: ("lookup_join_const_col_@12") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' │ - └── • scan buffer - columns: (column9, column1, column2, column10, check1) - label: buffer 1 + └── • project + │ columns: (column9, column2) + │ estimated row count: 2 + │ + └── • scan buffer + columns: (column9, column1, column2, column10, check1) + label: buffer 1 # Test that we use the index when available for de-duplicating INSERT ON # CONFLICT DO NOTHING rows before inserting. @@ -1025,7 +1029,6 @@ vectorized: true │ filter: column3 > 0 │ └── • scan buffer - estimated row count: 2 label: buffer 1 # No need to plan checks for a since it's always null. @@ -1067,7 +1070,6 @@ vectorized: true │ filter: column3 > 0 │ └── • scan buffer - estimated row count: 2 label: buffer 1 # Use all the unique indexes and constraints as arbiters for DO NOTHING with no @@ -1339,38 +1341,42 @@ vectorized: true │ columns: () │ └── • project - │ columns: (column1, column2, column3, column4) + │ columns: (column3) │ estimated row count: 1 │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@13", column1, column2, column3, column4) - │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) - │ equality: (lookup_join_const_col_@13, column3) = (r,b) - │ pred: (column1 != r) OR (column2 != a) + └── • project + │ columns: (column1, column2, column3, column4) + │ estimated row count: 1 │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: ("lookup_join_const_col_@13", column1, column2, column3, column4) - │ estimated row count: 6 - │ - ├── • values - │ columns: ("lookup_join_const_col_@13") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) + │ equality: (lookup_join_const_col_@13, column3) = (r,b) + │ pred: (column1 != r) OR (column2 != a) │ - └── • filter - │ columns: (column1, column2, column3, column4) - │ estimated row count: 2 - │ filter: column4 IN ('bar', 'baz', 'foo') + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@13", column1, column2, column3, column4) + │ estimated row count: 6 │ - └── • project + ├── • values + │ columns: ("lookup_join_const_col_@13") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' + │ + └── • filter │ columns: (column1, column2, column3, column4) │ estimated row count: 2 + │ filter: column4 IN ('bar', 'baz', 'foo') │ - └── • scan buffer - columns: (column1, column2, column3, column4, check1, partial_index_put1) - label: buffer 1 + └── • project + │ columns: (column1, column2, column3, column4) + │ estimated row count: 2 + │ + └── • scan buffer + columns: (column1, column2, column3, column4, check1, partial_index_put1) + label: buffer 1 # Test that we use the partial index when available for de-duplicating INSERT ON # CONFLICT DO NOTHING rows before inserting. @@ -1477,7 +1483,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 2 label: buffer 1 # By default, we do not require checks on UUID columns set to gen_random_uuid(), @@ -1514,7 +1519,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 1 label: buffer 1 # The default value of id1 is gen_random_uuid(), so we don't need to plan checks @@ -1552,7 +1556,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 1 label: buffer 1 # We can also detect gen_random_uuid() when it is a projection. @@ -1629,7 +1632,6 @@ vectorized: true │ │ spans: FULL SCAN │ │ │ └── • scan buffer -│ estimated row count: 1 │ label: buffer 1 │ └── • constraint-check @@ -1647,7 +1649,6 @@ vectorized: true │ spans: FULL SCAN │ └── • scan buffer - estimated row count: 1 label: buffer 1 statement ok @@ -2194,34 +2195,38 @@ vectorized: true │ │ columns: () │ │ │ └── • project -│ │ columns: (r_new, s_new, i_new, j) +│ │ columns: (i_new) │ │ estimated row count: 3 (missing stats) │ │ -│ └── • lookup join (semi) -│ │ columns: (r_new, s_new, i_new, j, "lookup_join_const_col_@17") -│ │ table: uniq_enum@primary -│ │ equality: (lookup_join_const_col_@17, i_new) = (r,i) -│ │ equality cols are key -│ │ pred: r_new != r +│ └── • project +│ │ columns: (r_new, i_new) +│ │ estimated row count: 3 (missing stats) │ │ -│ └── • cross join (inner) -│ │ columns: (r_new, s_new, i_new, j, "lookup_join_const_col_@17") -│ │ estimated row count: 30 (missing stats) -│ │ -│ ├── • project -│ │ │ columns: (r_new, s_new, i_new, j) -│ │ │ estimated row count: 10 (missing stats) -│ │ │ -│ │ └── • scan buffer -│ │ columns: (r, s, i, j, r_new, s_new, i_new, check1) -│ │ label: buffer 1 +│ └── • lookup join (semi) +│ │ columns: (r_new, i_new, "lookup_join_const_col_@17") +│ │ table: uniq_enum@primary +│ │ equality: (lookup_join_const_col_@17, i_new) = (r,i) +│ │ equality cols are key +│ │ pred: r_new != r │ │ -│ └── • values -│ columns: ("lookup_join_const_col_@17") -│ size: 1 column, 3 rows -│ row 0, expr 0: 'us-east' -│ row 1, expr 0: 'us-west' -│ row 2, expr 0: 'eu-west' +│ └── • cross join (inner) +│ │ columns: (r_new, i_new, "lookup_join_const_col_@17") +│ │ estimated row count: 30 (missing stats) +│ │ +│ ├── • project +│ │ │ columns: (r_new, i_new) +│ │ │ estimated row count: 10 (missing stats) +│ │ │ +│ │ └── • scan buffer +│ │ columns: (r, s, i, j, r_new, s_new, i_new, check1) +│ │ label: buffer 1 +│ │ +│ └── • values +│ columns: ("lookup_join_const_col_@17") +│ size: 1 column, 3 rows +│ row 0, expr 0: 'us-east' +│ row 1, expr 0: 'us-west' +│ row 2, expr 0: 'eu-west' │ └── • constraint-check │ @@ -2229,34 +2234,38 @@ vectorized: true │ columns: () │ └── • project - │ columns: (r_new, s_new, i_new, j) + │ columns: (s_new, j) │ estimated row count: 3 (missing stats) │ - └── • lookup join (semi) - │ columns: (r_new, s_new, i_new, j, "lookup_join_const_col_@27") - │ table: uniq_enum@uniq_enum_r_s_j_key - │ equality: (lookup_join_const_col_@27, s_new, j) = (r,s,j) - │ equality cols are key - │ pred: (r_new != r) OR (i_new != i) + └── • project + │ columns: (r_new, s_new, i_new, j) + │ estimated row count: 3 (missing stats) │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: (r_new, s_new, i_new, j, "lookup_join_const_col_@27") - │ estimated row count: 30 (missing stats) - │ - ├── • project - │ │ columns: (r_new, s_new, i_new, j) - │ │ estimated row count: 10 (missing stats) - │ │ - │ └── • scan buffer - │ columns: (r, s, i, j, r_new, s_new, i_new, check1) - │ label: buffer 1 + │ table: uniq_enum@uniq_enum_r_s_j_key + │ equality: (lookup_join_const_col_@27, s_new, j) = (r,s,j) + │ equality cols are key + │ pred: (r_new != r) OR (i_new != i) │ - └── • values - columns: ("lookup_join_const_col_@27") - size: 1 column, 3 rows - row 0, expr 0: 'us-east' - row 1, expr 0: 'us-west' - row 2, expr 0: 'eu-west' + └── • cross join (inner) + │ columns: (r_new, s_new, i_new, j, "lookup_join_const_col_@27") + │ estimated row count: 30 (missing stats) + │ + ├── • project + │ │ columns: (r_new, s_new, i_new, j) + │ │ estimated row count: 10 (missing stats) + │ │ + │ └── • scan buffer + │ columns: (r, s, i, j, r_new, s_new, i_new, check1) + │ label: buffer 1 + │ + └── • values + columns: ("lookup_join_const_col_@27") + size: 1 column, 3 rows + row 0, expr 0: 'us-east' + row 1, expr 0: 'us-west' + row 2, expr 0: 'eu-west' # None of the updated values have nulls. query T @@ -2508,38 +2517,42 @@ vectorized: true │ columns: () │ └── • project - │ columns: (r, a, b_new, c) + │ columns: (b_new) │ estimated row count: 0 │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@16", r, a, b_new, c) - │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) - │ equality: (lookup_join_const_col_@16, b_new) = (r,b) - │ pred: (r != r) OR (a != a) + └── • project + │ columns: (r, a, b_new, c) + │ estimated row count: 0 │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: ("lookup_join_const_col_@16", r, a, b_new, c) - │ estimated row count: 3 - │ - ├── • values - │ columns: ("lookup_join_const_col_@16") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) + │ equality: (lookup_join_const_col_@16, b_new) = (r,b) + │ pred: (r != r) OR (a != a) │ - └── • filter - │ columns: (r, a, b_new, c) - │ estimated row count: 1 - │ filter: c IN ('bar', 'baz', 'foo') + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@16", r, a, b_new, c) + │ estimated row count: 3 │ - └── • project + ├── • values + │ columns: ("lookup_join_const_col_@16") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' + │ + └── • filter │ columns: (r, a, b_new, c) │ estimated row count: 1 + │ filter: c IN ('bar', 'baz', 'foo') │ - └── • scan buffer - columns: (r, a, b, b_new, partial_index_put1, partial_index_put1, c) - label: buffer 1 + └── • project + │ columns: (r, a, b_new, c) + │ estimated row count: 1 + │ + └── • scan buffer + columns: (r, a, b, b_new, partial_index_put1, partial_index_put1, c) + label: buffer 1 # By default, we do not require checks on UUID columns set to gen_random_uuid(), # but we do for UUID columns set to other values. @@ -3399,34 +3412,38 @@ vectorized: true │ │ columns: () │ │ │ └── • project -│ │ columns: (upsert_r, column2, upsert_i, column4) +│ │ columns: (upsert_i) │ │ estimated row count: 1 (missing stats) │ │ -│ └── • lookup join (semi) -│ │ columns: ("lookup_join_const_col_@20", upsert_r, column2, upsert_i, column4) -│ │ table: uniq_enum@primary -│ │ equality: (lookup_join_const_col_@20, upsert_i) = (r,i) -│ │ equality cols are key -│ │ pred: upsert_r != r +│ └── • project +│ │ columns: (upsert_r, upsert_i) +│ │ estimated row count: 1 (missing stats) │ │ -│ └── • cross join (inner) -│ │ columns: ("lookup_join_const_col_@20", upsert_r, column2, upsert_i, column4) -│ │ estimated row count: 6 (missing stats) -│ │ -│ ├── • values -│ │ columns: ("lookup_join_const_col_@20") -│ │ size: 1 column, 3 rows -│ │ row 0, expr 0: 'us-east' -│ │ row 1, expr 0: 'us-west' -│ │ row 2, expr 0: 'eu-west' +│ └── • lookup join (semi) +│ │ columns: ("lookup_join_const_col_@20", upsert_r, upsert_i) +│ │ table: uniq_enum@primary +│ │ equality: (lookup_join_const_col_@20, upsert_i) = (r,i) +│ │ equality cols are key +│ │ pred: upsert_r != r │ │ -│ └── • project -│ │ columns: (upsert_r, column2, upsert_i, column4) -│ │ estimated row count: 2 (missing stats) +│ └── • cross join (inner) +│ │ columns: ("lookup_join_const_col_@20", upsert_r, upsert_i) +│ │ estimated row count: 6 (missing stats) +│ │ +│ ├── • values +│ │ columns: ("lookup_join_const_col_@20") +│ │ size: 1 column, 3 rows +│ │ row 0, expr 0: 'us-east' +│ │ row 1, expr 0: 'us-west' +│ │ row 2, expr 0: 'eu-west' │ │ -│ └── • scan buffer -│ columns: (column1, column2, column3, column4, r, s, i, j, column2, column4, r, check1, upsert_r, upsert_i) -│ label: buffer 1 +│ └── • project +│ │ columns: (upsert_r, upsert_i) +│ │ estimated row count: 2 (missing stats) +│ │ +│ └── • scan buffer +│ columns: (column1, column2, column3, column4, r, s, i, j, column2, column4, r, check1, upsert_r, upsert_i) +│ label: buffer 1 │ └── • constraint-check │ @@ -3434,34 +3451,38 @@ vectorized: true │ columns: () │ └── • project - │ columns: (upsert_r, column2, upsert_i, column4) + │ columns: (column2, column4) │ estimated row count: 1 (missing stats) │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@30", upsert_r, column2, upsert_i, column4) - │ table: uniq_enum@uniq_enum_r_s_j_key - │ equality: (lookup_join_const_col_@30, column2, column4) = (r,s,j) - │ equality cols are key - │ pred: (upsert_r != r) OR (upsert_i != i) + └── • project + │ columns: (upsert_r, column2, upsert_i, column4) + │ estimated row count: 1 (missing stats) │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: ("lookup_join_const_col_@30", upsert_r, column2, upsert_i, column4) - │ estimated row count: 6 (missing stats) - │ - ├── • values - │ columns: ("lookup_join_const_col_@30") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + │ table: uniq_enum@uniq_enum_r_s_j_key + │ equality: (lookup_join_const_col_@30, column2, column4) = (r,s,j) + │ equality cols are key + │ pred: (upsert_r != r) OR (upsert_i != i) │ - └── • project - │ columns: (upsert_r, column2, upsert_i, column4) - │ estimated row count: 2 (missing stats) + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@30", upsert_r, column2, upsert_i, column4) + │ estimated row count: 6 (missing stats) │ - └── • scan buffer - columns: (column1, column2, column3, column4, r, s, i, j, column2, column4, r, check1, upsert_r, upsert_i) - label: buffer 1 + ├── • values + │ columns: ("lookup_join_const_col_@30") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' + │ + └── • project + │ columns: (upsert_r, column2, upsert_i, column4) + │ estimated row count: 2 (missing stats) + │ + └── • scan buffer + columns: (column1, column2, column3, column4, r, s, i, j, column2, column4, r, check1, upsert_r, upsert_i) + label: buffer 1 # Test that we use the index when available for the ON CONFLICT checks. query T @@ -3481,14 +3502,14 @@ vectorized: true │ │ arbiter constraints: unique_s_j │ │ │ └── • buffer -│ │ columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r, upsert_s, upsert_j) +│ │ columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r) │ │ label: buffer 1 │ │ │ └── • project -│ │ columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r, upsert_s, upsert_j) +│ │ columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r) │ │ │ └── • render -│ │ columns: (check1, column1, column2, column3, column4, r, s, i, j, upsert_r, upsert_s, upsert_i, upsert_j) +│ │ columns: (check1, column1, column2, column3, column4, r, s, i, j, upsert_r, upsert_i) │ │ estimated row count: 2 (missing stats) │ │ render check1: upsert_r IN ('us-east', 'us-west', 'eu-west') │ │ render column1: column1 @@ -3500,17 +3521,13 @@ vectorized: true │ │ render i: i │ │ render j: j │ │ render upsert_r: upsert_r -│ │ render upsert_s: upsert_s │ │ render upsert_i: upsert_i -│ │ render upsert_j: upsert_j │ │ │ └── • render -│ │ columns: (upsert_r, upsert_s, upsert_i, upsert_j, column1, column2, column3, column4, r, s, i, j) +│ │ columns: (upsert_r, upsert_i, column1, column2, column3, column4, r, s, i, j) │ │ estimated row count: 2 (missing stats) │ │ render upsert_r: CASE WHEN r IS NULL THEN column1 ELSE r END -│ │ render upsert_s: CASE WHEN r IS NULL THEN column2 ELSE s END │ │ render upsert_i: CASE WHEN r IS NULL THEN column3 ELSE 3 END -│ │ render upsert_j: CASE WHEN r IS NULL THEN column4 ELSE j END │ │ render column1: column1 │ │ render column2: column2 │ │ render column3: column3 @@ -3545,34 +3562,38 @@ vectorized: true │ columns: () │ └── • project - │ columns: (upsert_r, upsert_s, upsert_i, upsert_j) + │ columns: (upsert_i) │ estimated row count: 1 (missing stats) │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@23", upsert_r, upsert_s, upsert_i, upsert_j) - │ table: uniq_enum@primary - │ equality: (lookup_join_const_col_@23, upsert_i) = (r,i) - │ equality cols are key - │ pred: upsert_r != r + └── • project + │ columns: (upsert_r, upsert_i) + │ estimated row count: 1 (missing stats) │ - └── • cross join (inner) - │ columns: ("lookup_join_const_col_@23", upsert_r, upsert_s, upsert_i, upsert_j) - │ estimated row count: 6 (missing stats) - │ - ├── • values - │ columns: ("lookup_join_const_col_@23") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + └── • lookup join (semi) + │ columns: ("lookup_join_const_col_@23", upsert_r, upsert_i) + │ table: uniq_enum@primary + │ equality: (lookup_join_const_col_@23, upsert_i) = (r,i) + │ equality cols are key + │ pred: upsert_r != r │ - └── • project - │ columns: (upsert_r, upsert_s, upsert_i, upsert_j) - │ estimated row count: 2 (missing stats) + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@23", upsert_r, upsert_i) + │ estimated row count: 6 (missing stats) + │ + ├── • values + │ columns: ("lookup_join_const_col_@23") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' │ - └── • scan buffer - columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r, upsert_s, upsert_j) - label: buffer 1 + └── • project + │ columns: (upsert_r, upsert_i) + │ estimated row count: 2 (missing stats) + │ + └── • scan buffer + columns: (column1, column2, column3, column4, r, s, i, j, upsert_i, r, check1, upsert_r) + label: buffer 1 # None of the upserted values have nulls. query T @@ -4069,38 +4090,42 @@ vectorized: true │ columns: () │ └── • project - │ columns: (upsert_r, upsert_a, column3, column4) + │ columns: (column3) │ estimated row count: 1 │ - └── • lookup join (semi) - │ columns: ("lookup_join_const_col_@22", upsert_r, upsert_a, column3, column4) - │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) - │ equality: (lookup_join_const_col_@22, column3) = (r,b) - │ pred: (upsert_r != r) OR (upsert_a != a) + └── • project + │ columns: (upsert_r, upsert_a, column3, column4) + │ estimated row count: 1 │ - └── • cross join (inner) + └── • lookup join (semi) │ columns: ("lookup_join_const_col_@22", upsert_r, upsert_a, column3, column4) - │ estimated row count: 6 - │ - ├── • values - │ columns: ("lookup_join_const_col_@22") - │ size: 1 column, 3 rows - │ row 0, expr 0: 'us-east' - │ row 1, expr 0: 'us-west' - │ row 2, expr 0: 'eu-west' + │ table: uniq_partial_enum@uniq_partial_enum_r_b_idx (partial index) + │ equality: (lookup_join_const_col_@22, column3) = (r,b) + │ pred: (upsert_r != r) OR (upsert_a != a) │ - └── • filter - │ columns: (upsert_r, upsert_a, column3, column4) - │ estimated row count: 2 - │ filter: column4 IN ('bar', 'baz', 'foo') + └── • cross join (inner) + │ columns: ("lookup_join_const_col_@22", upsert_r, upsert_a, column3, column4) + │ estimated row count: 6 │ - └── • project + ├── • values + │ columns: ("lookup_join_const_col_@22") + │ size: 1 column, 3 rows + │ row 0, expr 0: 'us-east' + │ row 1, expr 0: 'us-west' + │ row 2, expr 0: 'eu-west' + │ + └── • filter │ columns: (upsert_r, upsert_a, column3, column4) │ estimated row count: 2 + │ filter: column4 IN ('bar', 'baz', 'foo') │ - └── • scan buffer - columns: (column1, column2, column3, column4, r, a, b, c, column3, column4, r, check1, partial_index_put1, partial_index_del1, upsert_r, upsert_a) - label: buffer 1 + └── • project + │ columns: (upsert_r, upsert_a, column3, column4) + │ estimated row count: 2 + │ + └── • scan buffer + columns: (column1, column2, column3, column4, r, a, b, c, column3, column4, r, check1, partial_index_put1, partial_index_del1, upsert_r, upsert_a) + label: buffer 1 # Test that we use the partial index when available for de-duplicating INSERT ON # CONFLICT DO UPDATE rows before inserting. diff --git a/pkg/sql/opt/norm/testdata/rules/prune_cols b/pkg/sql/opt/norm/testdata/rules/prune_cols index baac757e9e03..d377d43f83e3 100644 --- a/pkg/sql/opt/norm/testdata/rules/prune_cols +++ b/pkg/sql/opt/norm/testdata/rules/prune_cols @@ -3592,8 +3592,6 @@ upsert checks └── column2:7 > upsert_b:16 [as=check3:19, outer=(7,16)] # Do not prune columns from updates that are needed for unique checks. -# TODO(mgartner): v and z can be pruned because they are not updated and not -# needed for uniqueness checks. norm expect=PruneMutationInputCols UPDATE uniq SET w = 1, x = 2 WHERE k = 3 ---- @@ -3607,19 +3605,19 @@ update uniq ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: w_new:15!null x_new:16!null uniq.k:8!null uniq.v:9 uniq.w:10 uniq.x:11 uniq.y:12 uniq.z:13 + │ ├── columns: w_new:15!null x_new:16!null uniq.k:8!null uniq.w:10 uniq.x:11 uniq.y:12 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(8-13,15,16) + │ ├── fd: ()-->(8,10-12,15,16) │ ├── select - │ │ ├── columns: uniq.k:8!null uniq.v:9 uniq.w:10 uniq.x:11 uniq.y:12 uniq.z:13 + │ │ ├── columns: uniq.k:8!null uniq.w:10 uniq.x:11 uniq.y:12 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(8-13) + │ │ ├── fd: ()-->(8,10-12) │ │ ├── scan uniq - │ │ │ ├── columns: uniq.k:8!null uniq.v:9 uniq.w:10 uniq.x:11 uniq.y:12 uniq.z:13 + │ │ │ ├── columns: uniq.k:8!null uniq.w:10 uniq.x:11 uniq.y:12 │ │ │ ├── key: (8) - │ │ │ └── fd: (8)-->(9-13), (13)~~>(8-12), (10)~~>(8,9,11-13), (11,12)~~>(8-10,13) + │ │ │ └── fd: (8)-->(10-12), (10)~~>(8,11,12), (11,12)~~>(8,10) │ │ └── filters │ │ └── uniq.k:8 = 3 [outer=(8), constraints=(/8: [/3 - /3]; tight), fd=()-->(8)] │ └── projections @@ -3627,60 +3625,61 @@ update uniq │ └── 2 [as=x_new:16] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:24!null v:25 w:26!null x:27!null y:28 z:29 + │ └── project + │ ├── columns: w:26!null │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(24-29) - │ ├── with-scan &1 - │ │ ├── columns: k:24!null v:25 w:26!null x:27!null y:28 z:29 - │ │ ├── mapping: - │ │ │ ├── uniq.k:8 => k:24 - │ │ │ ├── uniq.v:9 => v:25 - │ │ │ ├── w_new:15 => w:26 - │ │ │ ├── x_new:16 => x:27 - │ │ │ ├── uniq.y:12 => y:28 - │ │ │ └── uniq.z:13 => z:29 - │ │ ├── cardinality: [0 - 1] - │ │ ├── key: () - │ │ └── fd: ()-->(24-29) - │ ├── scan uniq - │ │ ├── columns: uniq.k:17!null uniq.w:19 - │ │ ├── key: (17) - │ │ └── fd: (17)-->(19) - │ └── filters - │ ├── w:26 = uniq.w:19 [outer=(19,26), constraints=(/19: (/NULL - ]; /26: (/NULL - ]), fd=(19)==(26), (26)==(19)] - │ └── k:24 != uniq.k:17 [outer=(17,24), constraints=(/17: (/NULL - ]; /24: (/NULL - ])] + │ ├── fd: ()-->(26) + │ └── semi-join (hash) + │ ├── columns: k:24!null w:26!null + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(24,26) + │ ├── with-scan &1 + │ │ ├── columns: k:24!null w:26!null + │ │ ├── mapping: + │ │ │ ├── uniq.k:8 => k:24 + │ │ │ └── w_new:15 => w:26 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ └── fd: ()-->(24,26) + │ ├── scan uniq + │ │ ├── columns: uniq.k:17!null uniq.w:19 + │ │ ├── key: (17) + │ │ └── fd: (17)-->(19) + │ └── filters + │ ├── w:26 = uniq.w:19 [outer=(19,26), constraints=(/19: (/NULL - ]; /26: (/NULL - ]), fd=(19)==(26), (26)==(19)] + │ └── k:24 != uniq.k:17 [outer=(17,24), constraints=(/17: (/NULL - ]; /24: (/NULL - ])] └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:37!null v:38 w:39!null x:40!null y:41 z:42 + └── project + ├── columns: x:40!null y:41 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(37-42) - ├── with-scan &1 - │ ├── columns: k:37!null v:38 w:39!null x:40!null y:41 z:42 - │ ├── mapping: - │ │ ├── uniq.k:8 => k:37 - │ │ ├── uniq.v:9 => v:38 - │ │ ├── w_new:15 => w:39 - │ │ ├── x_new:16 => x:40 - │ │ ├── uniq.y:12 => y:41 - │ │ └── uniq.z:13 => z:42 - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ └── fd: ()-->(37-42) - ├── scan uniq - │ ├── columns: uniq.k:30!null uniq.x:33 uniq.y:34 - │ ├── key: (30) - │ └── fd: (30)-->(33,34) - └── filters - ├── x:40 = uniq.x:33 [outer=(33,40), constraints=(/33: (/NULL - ]; /40: (/NULL - ]), fd=(33)==(40), (40)==(33)] - ├── y:41 = uniq.y:34 [outer=(34,41), constraints=(/34: (/NULL - ]; /41: (/NULL - ]), fd=(34)==(41), (41)==(34)] - └── k:37 != uniq.k:30 [outer=(30,37), constraints=(/30: (/NULL - ]; /37: (/NULL - ])] + ├── fd: ()-->(40,41) + └── semi-join (hash) + ├── columns: k:37!null x:40!null y:41 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(37,40,41) + ├── with-scan &1 + │ ├── columns: k:37!null x:40!null y:41 + │ ├── mapping: + │ │ ├── uniq.k:8 => k:37 + │ │ ├── x_new:16 => x:40 + │ │ └── uniq.y:12 => y:41 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ └── fd: ()-->(37,40,41) + ├── scan uniq + │ ├── columns: uniq.k:30!null uniq.x:33 uniq.y:34 + │ ├── key: (30) + │ └── fd: (30)-->(33,34) + └── filters + ├── x:40 = uniq.x:33 [outer=(33,40), constraints=(/33: (/NULL - ]; /40: (/NULL - ]), fd=(33)==(40), (40)==(33)] + ├── y:41 = uniq.y:34 [outer=(34,41), constraints=(/34: (/NULL - ]; /41: (/NULL - ]), fd=(34)==(41), (41)==(34)] + └── k:37 != uniq.k:30 [outer=(30,37), constraints=(/30: (/NULL - ]; /37: (/NULL - ])] # Do not prune columns from updates that are needed for partial unique checks. -# TODO(mgartner): x can be pruned because it is not updated and not needed for -# uniqueness checks. norm expect=PruneMutationInputCols UPDATE uniq_partial SET v = 1 WHERE k = 3 ---- @@ -3693,60 +3692,64 @@ update uniq_partial ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: v_new:11!null uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 uniq_partial.x:9 + │ ├── columns: v_new:11!null uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(6-9,11) + │ ├── fd: ()-->(6-8,11) │ ├── select - │ │ ├── columns: uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 uniq_partial.x:9 + │ │ ├── columns: uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 │ │ ├── cardinality: [0 - 1] │ │ ├── key: () - │ │ ├── fd: ()-->(6-9) + │ │ ├── fd: ()-->(6-8) │ │ ├── scan uniq_partial - │ │ │ ├── columns: uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 uniq_partial.x:9 + │ │ │ ├── columns: uniq_partial.k:6!null uniq_partial.v:7 uniq_partial.w:8 │ │ │ ├── key: (6) - │ │ │ └── fd: (6)-->(7-9) + │ │ │ └── fd: (6)-->(7,8) │ │ └── filters │ │ └── uniq_partial.k:6 = 3 [outer=(6), constraints=(/6: [/3 - /3]; tight), fd=()-->(6)] │ └── projections │ └── 1 [as=v_new:11] └── unique-checks └── unique-checks-item: uniq_partial(v) - └── semi-join (hash) - ├── columns: k:17!null v:18!null w:19!null x:20 + └── project + ├── columns: v:18!null ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(17-20) - ├── select - │ ├── columns: k:17!null v:18!null w:19!null x:20 - │ ├── cardinality: [0 - 1] - │ ├── key: () - │ ├── fd: ()-->(17-20) - │ ├── with-scan &1 - │ │ ├── columns: k:17!null v:18!null w:19 x:20 - │ │ ├── mapping: - │ │ │ ├── uniq_partial.k:6 => k:17 - │ │ │ ├── v_new:11 => v:18 - │ │ │ ├── uniq_partial.w:8 => w:19 - │ │ │ └── uniq_partial.x:9 => x:20 - │ │ ├── cardinality: [0 - 1] - │ │ ├── key: () - │ │ └── fd: ()-->(17-20) - │ └── filters - │ └── w:19 > 0 [outer=(19), constraints=(/19: [/1 - ]; tight)] - ├── select - │ ├── columns: uniq_partial.k:12!null uniq_partial.v:13 uniq_partial.w:14!null - │ ├── key: (12) - │ ├── fd: (12)-->(13,14) - │ ├── scan uniq_partial - │ │ ├── columns: uniq_partial.k:12!null uniq_partial.v:13 uniq_partial.w:14 - │ │ ├── key: (12) - │ │ └── fd: (12)-->(13,14) - │ └── filters - │ └── uniq_partial.w:14 > 0 [outer=(14), constraints=(/14: [/1 - ]; tight)] - └── filters - ├── v:18 = uniq_partial.v:13 [outer=(13,18), constraints=(/13: (/NULL - ]; /18: (/NULL - ]), fd=(13)==(18), (18)==(13)] - └── k:17 != uniq_partial.k:12 [outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ])] + ├── fd: ()-->(18) + └── semi-join (hash) + ├── columns: k:17!null v:18!null w:19!null + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(17-19) + ├── select + │ ├── columns: k:17!null v:18!null w:19!null + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(17-19) + │ ├── with-scan &1 + │ │ ├── columns: k:17!null v:18!null w:19 + │ │ ├── mapping: + │ │ │ ├── uniq_partial.k:6 => k:17 + │ │ │ ├── v_new:11 => v:18 + │ │ │ └── uniq_partial.w:8 => w:19 + │ │ ├── cardinality: [0 - 1] + │ │ ├── key: () + │ │ └── fd: ()-->(17-19) + │ └── filters + │ └── w:19 > 0 [outer=(19), constraints=(/19: [/1 - ]; tight)] + ├── select + │ ├── columns: uniq_partial.k:12!null uniq_partial.v:13 uniq_partial.w:14!null + │ ├── key: (12) + │ ├── fd: (12)-->(13,14) + │ ├── scan uniq_partial + │ │ ├── columns: uniq_partial.k:12!null uniq_partial.v:13 uniq_partial.w:14 + │ │ ├── key: (12) + │ │ └── fd: (12)-->(13,14) + │ └── filters + │ └── uniq_partial.w:14 > 0 [outer=(14), constraints=(/14: [/1 - ]; tight)] + └── filters + ├── v:18 = uniq_partial.v:13 [outer=(13,18), constraints=(/13: (/NULL - ]; /18: (/NULL - ]), fd=(13)==(18), (18)==(13)] + └── k:17 != uniq_partial.k:12 [outer=(12,17), constraints=(/12: (/NULL - ]; /17: (/NULL - ])] # Do not prune columns that are needed for foreign key checks or cascades. norm expect=PruneMutationInputCols @@ -3771,16 +3774,16 @@ upsert uniq_fk_parent ├── cardinality: [0 - 0] ├── volatile, mutations ├── project - │ ├── columns: upsert_k:17 upsert_a:18 upsert_b:19 upsert_c:20 upsert_d:21 column1:7!null column2:8!null column9:9 uniq_fk_parent.k:10 uniq_fk_parent.b:12 uniq_fk_parent.c:13 + │ ├── columns: upsert_k:17 upsert_a:18 upsert_b:19 upsert_c:20 column1:7!null column2:8!null column9:9 uniq_fk_parent.k:10 uniq_fk_parent.b:12 uniq_fk_parent.c:13 │ ├── cardinality: [1 - 1] │ ├── key: () - │ ├── fd: ()-->(7-10,12,13,17-21) + │ ├── fd: ()-->(7-10,12,13,17-20) │ ├── left-join (cross) - │ │ ├── columns: column1:7!null column2:8!null column9:9 uniq_fk_parent.k:10 uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 uniq_fk_parent.d:14 + │ │ ├── columns: column1:7!null column2:8!null column9:9 uniq_fk_parent.k:10 uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 │ │ ├── cardinality: [1 - 1] │ │ ├── multiplicity: left-rows(exactly-one), right-rows(exactly-one) │ │ ├── key: () - │ │ ├── fd: ()-->(7-14) + │ │ ├── fd: ()-->(7-13) │ │ ├── values │ │ │ ├── columns: column1:7!null column2:8!null column9:9 │ │ │ ├── cardinality: [1 - 1] @@ -3788,14 +3791,14 @@ upsert uniq_fk_parent │ │ │ ├── fd: ()-->(7-9) │ │ │ └── (2, 1, NULL) │ │ ├── select - │ │ │ ├── columns: uniq_fk_parent.k:10!null uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 uniq_fk_parent.d:14 + │ │ │ ├── columns: uniq_fk_parent.k:10!null uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 │ │ │ ├── cardinality: [0 - 1] │ │ │ ├── key: () - │ │ │ ├── fd: ()-->(10-14) + │ │ │ ├── fd: ()-->(10-13) │ │ │ ├── scan uniq_fk_parent - │ │ │ │ ├── columns: uniq_fk_parent.k:10!null uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 uniq_fk_parent.d:14 + │ │ │ │ ├── columns: uniq_fk_parent.k:10!null uniq_fk_parent.a:11 uniq_fk_parent.b:12 uniq_fk_parent.c:13 │ │ │ │ ├── key: (10) - │ │ │ │ └── fd: (10)-->(11-14), (11)~~>(10,12-14), (12,13)~~>(10,11,14) + │ │ │ │ └── fd: (10)-->(11-13), (11)~~>(10,12,13), (12,13)~~>(10,11) │ │ │ └── filters │ │ │ └── uniq_fk_parent.k:10 = 2 [outer=(10), constraints=(/10: [/2 - /2]; tight), fd=()-->(10)] │ │ └── filters (true) @@ -3803,58 +3806,62 @@ upsert uniq_fk_parent │ ├── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column1:7 ELSE uniq_fk_parent.k:10 END [as=upsert_k:17, outer=(7,10)] │ ├── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column2:8 ELSE uniq_fk_parent.a:11 END [as=upsert_a:18, outer=(8,10,11)] │ ├── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column9:9 ELSE uniq_fk_parent.b:12 END [as=upsert_b:19, outer=(9,10,12)] - │ ├── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column9:9 ELSE 1 END [as=upsert_c:20, outer=(9,10)] - │ └── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column9:9 ELSE uniq_fk_parent.d:14 END [as=upsert_d:21, outer=(9,10,14)] + │ └── CASE WHEN uniq_fk_parent.k:10 IS NULL THEN column9:9 ELSE 1 END [as=upsert_c:20, outer=(9,10)] └── unique-checks ├── unique-checks-item: uniq_fk_parent(a) - │ └── semi-join (hash) - │ ├── columns: k:28 a:29 b:30 c:31 d:32 + │ └── project + │ ├── columns: a:29 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(28-32) - │ ├── with-scan &1 - │ │ ├── columns: k:28 a:29 b:30 c:31 d:32 - │ │ ├── mapping: - │ │ │ ├── upsert_k:17 => k:28 - │ │ │ ├── upsert_a:18 => a:29 - │ │ │ ├── upsert_b:19 => b:30 - │ │ │ ├── upsert_c:20 => c:31 - │ │ │ └── upsert_d:21 => d:32 - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── fd: ()-->(28-32) - │ ├── scan uniq_fk_parent - │ │ ├── columns: uniq_fk_parent.k:22!null uniq_fk_parent.a:23 - │ │ ├── key: (22) - │ │ └── fd: (22)-->(23) - │ └── filters - │ ├── a:29 = uniq_fk_parent.a:23 [outer=(23,29), constraints=(/23: (/NULL - ]; /29: (/NULL - ]), fd=(23)==(29), (29)==(23)] - │ └── k:28 != uniq_fk_parent.k:22 [outer=(22,28), constraints=(/22: (/NULL - ]; /28: (/NULL - ])] + │ ├── fd: ()-->(29) + │ └── semi-join (hash) + │ ├── columns: k:28 a:29 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(28,29) + │ ├── with-scan &1 + │ │ ├── columns: k:28 a:29 + │ │ ├── mapping: + │ │ │ ├── upsert_k:17 => k:28 + │ │ │ └── upsert_a:18 => a:29 + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── fd: ()-->(28,29) + │ ├── scan uniq_fk_parent + │ │ ├── columns: uniq_fk_parent.k:22!null uniq_fk_parent.a:23 + │ │ ├── key: (22) + │ │ └── fd: (22)-->(23) + │ └── filters + │ ├── a:29 = uniq_fk_parent.a:23 [outer=(23,29), constraints=(/23: (/NULL - ]; /29: (/NULL - ]), fd=(23)==(29), (29)==(23)] + │ └── k:28 != uniq_fk_parent.k:22 [outer=(22,28), constraints=(/22: (/NULL - ]; /28: (/NULL - ])] └── unique-checks-item: uniq_fk_parent(b,c) - └── semi-join (hash) - ├── columns: k:39 a:40 b:41 c:42 d:43 + └── project + ├── columns: b:41 c:42 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(39-43) - ├── with-scan &1 - │ ├── columns: k:39 a:40 b:41 c:42 d:43 - │ ├── mapping: - │ │ ├── upsert_k:17 => k:39 - │ │ ├── upsert_a:18 => a:40 - │ │ ├── upsert_b:19 => b:41 - │ │ ├── upsert_c:20 => c:42 - │ │ └── upsert_d:21 => d:43 - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── fd: ()-->(39-43) - ├── scan uniq_fk_parent - │ ├── columns: uniq_fk_parent.k:33!null uniq_fk_parent.b:35 uniq_fk_parent.c:36 - │ ├── key: (33) - │ └── fd: (33)-->(35,36) - └── filters - ├── b:41 = uniq_fk_parent.b:35 [outer=(35,41), constraints=(/35: (/NULL - ]; /41: (/NULL - ]), fd=(35)==(41), (41)==(35)] - ├── c:42 = uniq_fk_parent.c:36 [outer=(36,42), constraints=(/36: (/NULL - ]; /42: (/NULL - ]), fd=(36)==(42), (42)==(36)] - └── k:39 != uniq_fk_parent.k:33 [outer=(33,39), constraints=(/33: (/NULL - ]; /39: (/NULL - ])] + ├── fd: ()-->(41,42) + └── semi-join (hash) + ├── columns: k:39 b:41 c:42 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(39,41,42) + ├── with-scan &1 + │ ├── columns: k:39 b:41 c:42 + │ ├── mapping: + │ │ ├── upsert_k:17 => k:39 + │ │ ├── upsert_b:19 => b:41 + │ │ └── upsert_c:20 => c:42 + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── fd: ()-->(39,41,42) + ├── scan uniq_fk_parent + │ ├── columns: uniq_fk_parent.k:33!null uniq_fk_parent.b:35 uniq_fk_parent.c:36 + │ ├── key: (33) + │ └── fd: (33)-->(35,36) + └── filters + ├── b:41 = uniq_fk_parent.b:35 [outer=(35,41), constraints=(/35: (/NULL - ]; /41: (/NULL - ]), fd=(35)==(41), (41)==(35)] + ├── c:42 = uniq_fk_parent.c:36 [outer=(36,42), constraints=(/36: (/NULL - ]; /42: (/NULL - ]), fd=(36)==(42), (42)==(36)] + └── k:39 != uniq_fk_parent.k:33 [outer=(33,39), constraints=(/33: (/NULL - ]; /39: (/NULL - ])] # Prune inbound foreign key columns when they are not updated. norm expect=PruneMutationInputCols @@ -3913,54 +3920,59 @@ upsert uniq_fk_parent │ └── CASE WHEN uniq_fk_parent.k:9 IS NULL THEN column8:8 ELSE 1 END [as=upsert_d:20, outer=(8,9)] └── unique-checks ├── unique-checks-item: uniq_fk_parent(a) - │ └── semi-join (hash) - │ ├── columns: k:27 a:28 b:29 c:30 d:31 + │ └── project + │ ├── columns: a:28 │ ├── cardinality: [0 - 1] │ ├── key: () - │ ├── fd: ()-->(27-31) - │ ├── with-scan &1 - │ │ ├── columns: k:27 a:28 b:29 c:30 d:31 - │ │ ├── mapping: - │ │ │ ├── upsert_k:16 => k:27 - │ │ │ ├── upsert_a:17 => a:28 - │ │ │ ├── upsert_b:18 => b:29 - │ │ │ ├── upsert_c:19 => c:30 - │ │ │ └── upsert_d:20 => d:31 - │ │ ├── cardinality: [1 - 1] - │ │ ├── key: () - │ │ └── fd: ()-->(27-31) - │ ├── scan uniq_fk_parent - │ │ ├── columns: uniq_fk_parent.k:21!null uniq_fk_parent.a:22 - │ │ ├── key: (21) - │ │ └── fd: (21)-->(22) - │ └── filters - │ ├── a:28 = uniq_fk_parent.a:22 [outer=(22,28), constraints=(/22: (/NULL - ]; /28: (/NULL - ]), fd=(22)==(28), (28)==(22)] - │ └── k:27 != uniq_fk_parent.k:21 [outer=(21,27), constraints=(/21: (/NULL - ]; /27: (/NULL - ])] + │ ├── fd: ()-->(28) + │ └── semi-join (hash) + │ ├── columns: k:27 a:28 + │ ├── cardinality: [0 - 1] + │ ├── key: () + │ ├── fd: ()-->(27,28) + │ ├── with-scan &1 + │ │ ├── columns: k:27 a:28 + │ │ ├── mapping: + │ │ │ ├── upsert_k:16 => k:27 + │ │ │ └── upsert_a:17 => a:28 + │ │ ├── cardinality: [1 - 1] + │ │ ├── key: () + │ │ └── fd: ()-->(27,28) + │ ├── scan uniq_fk_parent + │ │ ├── columns: uniq_fk_parent.k:21!null uniq_fk_parent.a:22 + │ │ ├── key: (21) + │ │ └── fd: (21)-->(22) + │ └── filters + │ ├── a:28 = uniq_fk_parent.a:22 [outer=(22,28), constraints=(/22: (/NULL - ]; /28: (/NULL - ]), fd=(22)==(28), (28)==(22)] + │ └── k:27 != uniq_fk_parent.k:21 [outer=(21,27), constraints=(/21: (/NULL - ]; /27: (/NULL - ])] └── unique-checks-item: uniq_fk_parent(b,c) - └── semi-join (hash) - ├── columns: k:38 a:39 b:40 c:41 d:42 + └── project + ├── columns: b:40 c:41 ├── cardinality: [0 - 1] ├── key: () - ├── fd: ()-->(38-42) - ├── with-scan &1 - │ ├── columns: k:38 a:39 b:40 c:41 d:42 - │ ├── mapping: - │ │ ├── upsert_k:16 => k:38 - │ │ ├── upsert_a:17 => a:39 - │ │ ├── upsert_b:18 => b:40 - │ │ ├── upsert_c:19 => c:41 - │ │ └── upsert_d:20 => d:42 - │ ├── cardinality: [1 - 1] - │ ├── key: () - │ └── fd: ()-->(38-42) - ├── scan uniq_fk_parent - │ ├── columns: uniq_fk_parent.k:32!null uniq_fk_parent.b:34 uniq_fk_parent.c:35 - │ ├── key: (32) - │ └── fd: (32)-->(34,35) - └── filters - ├── b:40 = uniq_fk_parent.b:34 [outer=(34,40), constraints=(/34: (/NULL - ]; /40: (/NULL - ]), fd=(34)==(40), (40)==(34)] - ├── c:41 = uniq_fk_parent.c:35 [outer=(35,41), constraints=(/35: (/NULL - ]; /41: (/NULL - ]), fd=(35)==(41), (41)==(35)] - └── k:38 != uniq_fk_parent.k:32 [outer=(32,38), constraints=(/32: (/NULL - ]; /38: (/NULL - ])] + ├── fd: ()-->(40,41) + └── semi-join (hash) + ├── columns: k:38 b:40 c:41 + ├── cardinality: [0 - 1] + ├── key: () + ├── fd: ()-->(38,40,41) + ├── with-scan &1 + │ ├── columns: k:38 b:40 c:41 + │ ├── mapping: + │ │ ├── upsert_k:16 => k:38 + │ │ ├── upsert_b:18 => b:40 + │ │ └── upsert_c:19 => c:41 + │ ├── cardinality: [1 - 1] + │ ├── key: () + │ └── fd: ()-->(38,40,41) + ├── scan uniq_fk_parent + │ ├── columns: uniq_fk_parent.k:32!null uniq_fk_parent.b:34 uniq_fk_parent.c:35 + │ ├── key: (32) + │ └── fd: (32)-->(34,35) + └── filters + ├── b:40 = uniq_fk_parent.b:34 [outer=(34,40), constraints=(/34: (/NULL - ]; /40: (/NULL - ]), fd=(34)==(40), (40)==(34)] + ├── c:41 = uniq_fk_parent.c:35 [outer=(35,41), constraints=(/35: (/NULL - ]; /41: (/NULL - ]), fd=(35)==(41), (41)==(35)] + └── k:38 != uniq_fk_parent.k:32 [outer=(32,38), constraints=(/32: (/NULL - ]; /38: (/NULL - ])] # Do not prune columns that are needed for foreign key checks or cascades. norm expect=PruneMutationInputCols diff --git a/pkg/sql/opt/optbuilder/mutation_builder_unique.go b/pkg/sql/opt/optbuilder/mutation_builder_unique.go index fa48ea750815..26c2e0c41cef 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder_unique.go +++ b/pkg/sql/opt/optbuilder/mutation_builder_unique.go @@ -151,7 +151,7 @@ func (mb *mutationBuilder) hasUniqueWithoutIndexConstraints() bool { // constraint are being updated (according to updateColIDs). When the unique // constraint has a partial predicate, it also returns true if the predicate // references any of the columns being updated. -func (mb *mutationBuilder) uniqueColsUpdated(uniqueOrdinal int) bool { +func (mb *mutationBuilder) uniqueColsUpdated(uniqueOrdinal cat.UniqueOrdinal) bool { uc := mb.tab.Unique(uniqueOrdinal) for i, n := 0, uc.ColumnCount(); i < n; i++ { @@ -379,7 +379,13 @@ func (h *uniqueCheckHelper) buildInsertionCheck() memo.UniqueChecksItem { keyCols = append(keyCols, withScanScope.cols[i].id) } - return f.ConstructUniqueChecksItem(semiJoin, &memo.UniqueChecksItemPrivate{ + // Create a Project that passes-through only the key columns. This allows + // normalization rules to prune any unnecessary columns from the expression. + // The key columns are always needed in order to display the constraint + // violation error. + project := f.ConstructProject(semiJoin, nil /* projections */, keyCols.ToSet()) + + return f.ConstructUniqueChecksItem(project, &memo.UniqueChecksItemPrivate{ Table: h.mb.tabID, CheckOrdinal: h.uniqueOrdinal, KeyCols: keyCols, diff --git a/pkg/sql/opt/optbuilder/testdata/unique-checks-insert b/pkg/sql/opt/optbuilder/testdata/unique-checks-insert index 80db059a957c..8356de16278d 100644 --- a/pkg/sql/opt/optbuilder/testdata/unique-checks-insert +++ b/pkg/sql/opt/optbuilder/testdata/unique-checks-insert @@ -28,38 +28,42 @@ insert uniq │ └── (2, 2, 2, 2, 2) └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22!null - │ ├── with-scan &1 - │ │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22!null - │ │ └── mapping: - │ │ ├── column1:7 => k:18 - │ │ ├── column2:8 => v:19 - │ │ ├── column3:9 => w:20 - │ │ ├── column4:10 => x:21 - │ │ └── column5:11 => y:22 - │ ├── scan uniq - │ │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 - │ └── filters - │ ├── w:20 = uniq.w:14 - │ └── k:18 != uniq.k:12 + │ └── project + │ ├── columns: w:20!null + │ └── semi-join (hash) + │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22!null + │ ├── with-scan &1 + │ │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22!null + │ │ └── mapping: + │ │ ├── column1:7 => k:18 + │ │ ├── column2:8 => v:19 + │ │ ├── column3:9 => w:20 + │ │ ├── column4:10 => x:21 + │ │ └── column5:11 => y:22 + │ ├── scan uniq + │ │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 + │ └── filters + │ ├── w:20 = uniq.w:14 + │ └── k:18 != uniq.k:12 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:29!null v:30!null w:31!null x:32!null y:33!null - ├── with-scan &1 - │ ├── columns: k:29!null v:30!null w:31!null x:32!null y:33!null - │ └── mapping: - │ ├── column1:7 => k:29 - │ ├── column2:8 => v:30 - │ ├── column3:9 => w:31 - │ ├── column4:10 => x:32 - │ └── column5:11 => y:33 - ├── scan uniq - │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 - └── filters - ├── x:32 = uniq.x:26 - ├── y:33 = uniq.y:27 - └── k:29 != uniq.k:23 + └── project + ├── columns: x:32!null y:33!null + └── semi-join (hash) + ├── columns: k:29!null v:30!null w:31!null x:32!null y:33!null + ├── with-scan &1 + │ ├── columns: k:29!null v:30!null w:31!null x:32!null y:33!null + │ └── mapping: + │ ├── column1:7 => k:29 + │ ├── column2:8 => v:30 + │ ├── column3:9 => w:31 + │ ├── column4:10 => x:32 + │ └── column5:11 => y:33 + ├── scan uniq + │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 + └── filters + ├── x:32 = uniq.x:26 + ├── y:33 = uniq.y:27 + └── k:29 != uniq.k:23 # Some of the inserted values have nulls. build @@ -81,38 +85,42 @@ insert uniq │ └── (3, NULL::INT8, NULL::INT8, NULL::INT8, 3) └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:18!null v:19 w:20 x:21 y:22!null - │ ├── with-scan &1 - │ │ ├── columns: k:18!null v:19 w:20 x:21 y:22!null - │ │ └── mapping: - │ │ ├── column1:7 => k:18 - │ │ ├── column2:8 => v:19 - │ │ ├── column3:9 => w:20 - │ │ ├── column4:10 => x:21 - │ │ └── column5:11 => y:22 - │ ├── scan uniq - │ │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 - │ └── filters - │ ├── w:20 = uniq.w:14 - │ └── k:18 != uniq.k:12 + │ └── project + │ ├── columns: w:20 + │ └── semi-join (hash) + │ ├── columns: k:18!null v:19 w:20 x:21 y:22!null + │ ├── with-scan &1 + │ │ ├── columns: k:18!null v:19 w:20 x:21 y:22!null + │ │ └── mapping: + │ │ ├── column1:7 => k:18 + │ │ ├── column2:8 => v:19 + │ │ ├── column3:9 => w:20 + │ │ ├── column4:10 => x:21 + │ │ └── column5:11 => y:22 + │ ├── scan uniq + │ │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 + │ └── filters + │ ├── w:20 = uniq.w:14 + │ └── k:18 != uniq.k:12 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:29!null v:30 w:31 x:32 y:33!null - ├── with-scan &1 - │ ├── columns: k:29!null v:30 w:31 x:32 y:33!null - │ └── mapping: - │ ├── column1:7 => k:29 - │ ├── column2:8 => v:30 - │ ├── column3:9 => w:31 - │ ├── column4:10 => x:32 - │ └── column5:11 => y:33 - ├── scan uniq - │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 - └── filters - ├── x:32 = uniq.x:26 - ├── y:33 = uniq.y:27 - └── k:29 != uniq.k:23 + └── project + ├── columns: x:32 y:33!null + └── semi-join (hash) + ├── columns: k:29!null v:30 w:31 x:32 y:33!null + ├── with-scan &1 + │ ├── columns: k:29!null v:30 w:31 x:32 y:33!null + │ └── mapping: + │ ├── column1:7 => k:29 + │ ├── column2:8 => v:30 + │ ├── column3:9 => w:31 + │ ├── column4:10 => x:32 + │ └── column5:11 => y:33 + ├── scan uniq + │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 + └── filters + ├── x:32 = uniq.x:26 + ├── y:33 = uniq.y:27 + └── k:29 != uniq.k:23 # No need to plan checks for w since it's always null. build @@ -133,22 +141,24 @@ insert uniq │ └── (2, 2, NULL::INT8, 2, 2) └── unique-checks └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:18!null v:19!null w:20 x:21!null y:22!null - ├── with-scan &1 - │ ├── columns: k:18!null v:19!null w:20 x:21!null y:22!null - │ └── mapping: - │ ├── column1:7 => k:18 - │ ├── column2:8 => v:19 - │ ├── column3:9 => w:20 - │ ├── column4:10 => x:21 - │ └── column5:11 => y:22 - ├── scan uniq - │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 - └── filters - ├── x:21 = uniq.x:15 - ├── y:22 = uniq.y:16 - └── k:18 != uniq.k:12 + └── project + ├── columns: x:21!null y:22!null + └── semi-join (hash) + ├── columns: k:18!null v:19!null w:20 x:21!null y:22!null + ├── with-scan &1 + │ ├── columns: k:18!null v:19!null w:20 x:21!null y:22!null + │ └── mapping: + │ ├── column1:7 => k:18 + │ ├── column2:8 => v:19 + │ ├── column3:9 => w:20 + │ ├── column4:10 => x:21 + │ └── column5:11 => y:22 + ├── scan uniq + │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 + └── filters + ├── x:21 = uniq.x:15 + ├── y:22 = uniq.y:16 + └── k:18 != uniq.k:12 # No need to plan checks for x,y since x is always null. build @@ -169,21 +179,23 @@ insert uniq │ └── (2, 2, NULL::INT8, NULL::INT8, 2) └── unique-checks └── unique-checks-item: uniq(w) - └── semi-join (hash) - ├── columns: k:18!null v:19!null w:20 x:21 y:22!null - ├── with-scan &1 - │ ├── columns: k:18!null v:19!null w:20 x:21 y:22!null - │ └── mapping: - │ ├── column1:7 => k:18 - │ ├── column2:8 => v:19 - │ ├── column3:9 => w:20 - │ ├── column4:10 => x:21 - │ └── column5:11 => y:22 - ├── scan uniq - │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 - └── filters - ├── w:20 = uniq.w:14 - └── k:18 != uniq.k:12 + └── project + ├── columns: w:20 + └── semi-join (hash) + ├── columns: k:18!null v:19!null w:20 x:21 y:22!null + ├── with-scan &1 + │ ├── columns: k:18!null v:19!null w:20 x:21 y:22!null + │ └── mapping: + │ ├── column1:7 => k:18 + │ ├── column2:8 => v:19 + │ ├── column3:9 => w:20 + │ ├── column4:10 => x:21 + │ └── column5:11 => y:22 + ├── scan uniq + │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 + └── filters + ├── w:20 = uniq.w:14 + └── k:18 != uniq.k:12 # No need to plan checks for x,y since y is always null. build @@ -204,21 +216,23 @@ insert uniq │ └── (2, 2, 2, 2, NULL::INT8) └── unique-checks └── unique-checks-item: uniq(w) - └── semi-join (hash) - ├── columns: k:18!null v:19!null w:20!null x:21!null y:22 - ├── with-scan &1 - │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22 - │ └── mapping: - │ ├── column1:7 => k:18 - │ ├── column2:8 => v:19 - │ ├── column3:9 => w:20 - │ ├── column4:10 => x:21 - │ └── column5:11 => y:22 - ├── scan uniq - │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 - └── filters - ├── w:20 = uniq.w:14 - └── k:18 != uniq.k:12 + └── project + ├── columns: w:20!null + └── semi-join (hash) + ├── columns: k:18!null v:19!null w:20!null x:21!null y:22 + ├── with-scan &1 + │ ├── columns: k:18!null v:19!null w:20!null x:21!null y:22 + │ └── mapping: + │ ├── column1:7 => k:18 + │ ├── column2:8 => v:19 + │ ├── column3:9 => w:20 + │ ├── column4:10 => x:21 + │ └── column5:11 => y:22 + ├── scan uniq + │ └── columns: uniq.k:12!null uniq.v:13 uniq.w:14 uniq.x:15 uniq.y:16 + └── filters + ├── w:20 = uniq.w:14 + └── k:18 != uniq.k:12 # No need to plan any checks, since w, x and y are always null. build @@ -364,22 +378,24 @@ insert uniq │ └── column5:11 └── unique-checks └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:24!null v:25!null w:26!null x:27!null y:28!null - ├── with-scan &1 - │ ├── columns: k:24!null v:25!null w:26!null x:27!null y:28!null - │ └── mapping: - │ ├── column1:7 => k:24 - │ ├── column2:8 => v:25 - │ ├── column3:9 => w:26 - │ ├── column4:10 => x:27 - │ └── column5:11 => y:28 - ├── scan uniq - │ └── columns: uniq.k:18!null uniq.v:19 uniq.w:20 uniq.x:21 uniq.y:22 - └── filters - ├── x:27 = uniq.x:21 - ├── y:28 = uniq.y:22 - └── k:24 != uniq.k:18 + └── project + ├── columns: x:27!null y:28!null + └── semi-join (hash) + ├── columns: k:24!null v:25!null w:26!null x:27!null y:28!null + ├── with-scan &1 + │ ├── columns: k:24!null v:25!null w:26!null x:27!null y:28!null + │ └── mapping: + │ ├── column1:7 => k:24 + │ ├── column2:8 => v:25 + │ ├── column3:9 => w:26 + │ ├── column4:10 => x:27 + │ └── column5:11 => y:28 + ├── scan uniq + │ └── columns: uniq.k:18!null uniq.v:19 uniq.w:20 uniq.x:21 uniq.y:22 + └── filters + ├── x:27 = uniq.x:21 + ├── y:28 = uniq.y:22 + └── k:24 != uniq.k:18 exec-ddl CREATE TABLE other (k INT, v INT, w INT NOT NULL, x INT, y INT) @@ -404,38 +420,42 @@ insert uniq │ └── columns: other.k:7 other.v:8 other.w:9!null other.x:10 other.y:11 rowid:12!null other.crdb_internal_mvcc_timestamp:13 └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:20 v:21 w:22!null x:23 y:24 - │ ├── with-scan &1 - │ │ ├── columns: k:20 v:21 w:22!null x:23 y:24 - │ │ └── mapping: - │ │ ├── other.k:7 => k:20 - │ │ ├── other.v:8 => v:21 - │ │ ├── other.w:9 => w:22 - │ │ ├── other.x:10 => x:23 - │ │ └── other.y:11 => y:24 - │ ├── scan uniq - │ │ └── columns: uniq.k:14!null uniq.v:15 uniq.w:16 uniq.x:17 uniq.y:18 - │ └── filters - │ ├── w:22 = uniq.w:16 - │ └── k:20 != uniq.k:14 + │ └── project + │ ├── columns: w:22!null + │ └── semi-join (hash) + │ ├── columns: k:20 v:21 w:22!null x:23 y:24 + │ ├── with-scan &1 + │ │ ├── columns: k:20 v:21 w:22!null x:23 y:24 + │ │ └── mapping: + │ │ ├── other.k:7 => k:20 + │ │ ├── other.v:8 => v:21 + │ │ ├── other.w:9 => w:22 + │ │ ├── other.x:10 => x:23 + │ │ └── other.y:11 => y:24 + │ ├── scan uniq + │ │ └── columns: uniq.k:14!null uniq.v:15 uniq.w:16 uniq.x:17 uniq.y:18 + │ └── filters + │ ├── w:22 = uniq.w:16 + │ └── k:20 != uniq.k:14 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:31 v:32 w:33!null x:34 y:35 - ├── with-scan &1 - │ ├── columns: k:31 v:32 w:33!null x:34 y:35 - │ └── mapping: - │ ├── other.k:7 => k:31 - │ ├── other.v:8 => v:32 - │ ├── other.w:9 => w:33 - │ ├── other.x:10 => x:34 - │ └── other.y:11 => y:35 - ├── scan uniq - │ └── columns: uniq.k:25!null uniq.v:26 uniq.w:27 uniq.x:28 uniq.y:29 - └── filters - ├── x:34 = uniq.x:28 - ├── y:35 = uniq.y:29 - └── k:31 != uniq.k:25 + └── project + ├── columns: x:34 y:35 + └── semi-join (hash) + ├── columns: k:31 v:32 w:33!null x:34 y:35 + ├── with-scan &1 + │ ├── columns: k:31 v:32 w:33!null x:34 y:35 + │ └── mapping: + │ ├── other.k:7 => k:31 + │ ├── other.v:8 => v:32 + │ ├── other.w:9 => w:33 + │ ├── other.x:10 => x:34 + │ └── other.y:11 => y:35 + ├── scan uniq + │ └── columns: uniq.k:25!null uniq.v:26 uniq.w:27 uniq.x:28 uniq.y:29 + └── filters + ├── x:34 = uniq.x:28 + ├── y:35 = uniq.y:29 + └── k:31 != uniq.k:25 exec-ddl CREATE TABLE uniq_overlaps_pk ( @@ -471,52 +491,58 @@ insert uniq_overlaps_pk │ └── (2, 2, 2, 2) └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ ├── with-scan &1 - │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ │ └── mapping: - │ │ ├── column1:6 => a:15 - │ │ ├── column2:7 => b:16 - │ │ ├── column3:8 => c:17 - │ │ └── column4:9 => d:18 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:10!null uniq_overlaps_pk.b:11!null uniq_overlaps_pk.c:12 uniq_overlaps_pk.d:13 - │ └── filters - │ ├── b:16 = uniq_overlaps_pk.b:11 - │ ├── c:17 = uniq_overlaps_pk.c:12 - │ └── a:15 != uniq_overlaps_pk.a:10 + │ └── project + │ ├── columns: b:16!null c:17!null + │ └── semi-join (hash) + │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ ├── with-scan &1 + │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ │ └── mapping: + │ │ ├── column1:6 => a:15 + │ │ ├── column2:7 => b:16 + │ │ ├── column3:8 => c:17 + │ │ └── column4:9 => d:18 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:10!null uniq_overlaps_pk.b:11!null uniq_overlaps_pk.c:12 uniq_overlaps_pk.d:13 + │ └── filters + │ ├── b:16 = uniq_overlaps_pk.b:11 + │ ├── c:17 = uniq_overlaps_pk.c:12 + │ └── a:15 != uniq_overlaps_pk.a:10 ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ ├── with-scan &1 - │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ │ └── mapping: - │ │ ├── column1:6 => a:24 - │ │ ├── column2:7 => b:25 - │ │ ├── column3:8 => c:26 - │ │ └── column4:9 => d:27 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:19!null uniq_overlaps_pk.b:20!null uniq_overlaps_pk.c:21 uniq_overlaps_pk.d:22 - │ └── filters - │ ├── a:24 = uniq_overlaps_pk.a:19 - │ └── b:25 != uniq_overlaps_pk.b:20 + │ └── project + │ ├── columns: a:24!null + │ └── semi-join (hash) + │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ ├── with-scan &1 + │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ │ └── mapping: + │ │ ├── column1:6 => a:24 + │ │ ├── column2:7 => b:25 + │ │ ├── column3:8 => c:26 + │ │ └── column4:9 => d:27 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:19!null uniq_overlaps_pk.b:20!null uniq_overlaps_pk.c:21 uniq_overlaps_pk.d:22 + │ └── filters + │ ├── a:24 = uniq_overlaps_pk.a:19 + │ └── b:25 != uniq_overlaps_pk.b:20 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:33!null b:34!null c:35!null d:36!null - ├── with-scan &1 - │ ├── columns: a:33!null b:34!null c:35!null d:36!null - │ └── mapping: - │ ├── column1:6 => a:33 - │ ├── column2:7 => b:34 - │ ├── column3:8 => c:35 - │ └── column4:9 => d:36 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:28!null uniq_overlaps_pk.b:29!null uniq_overlaps_pk.c:30 uniq_overlaps_pk.d:31 - └── filters - ├── c:35 = uniq_overlaps_pk.c:30 - ├── d:36 = uniq_overlaps_pk.d:31 - └── (a:33 != uniq_overlaps_pk.a:28) OR (b:34 != uniq_overlaps_pk.b:29) + └── project + ├── columns: c:35!null d:36!null + └── semi-join (hash) + ├── columns: a:33!null b:34!null c:35!null d:36!null + ├── with-scan &1 + │ ├── columns: a:33!null b:34!null c:35!null d:36!null + │ └── mapping: + │ ├── column1:6 => a:33 + │ ├── column2:7 => b:34 + │ ├── column3:8 => c:35 + │ └── column4:9 => d:36 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:28!null uniq_overlaps_pk.b:29!null uniq_overlaps_pk.c:30 uniq_overlaps_pk.d:31 + └── filters + ├── c:35 = uniq_overlaps_pk.c:30 + ├── d:36 = uniq_overlaps_pk.d:31 + └── (a:33 != uniq_overlaps_pk.a:28) OR (b:34 != uniq_overlaps_pk.b:29) # Insert with non-constant input. # Add inequality filters for the primary key columns that are not part of each @@ -538,52 +564,58 @@ insert uniq_overlaps_pk │ └── columns: k:6 v:7 w:8!null x:9 y:10 rowid:11!null other.crdb_internal_mvcc_timestamp:12 └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:18 b:19 c:20 d:21 - │ ├── with-scan &1 - │ │ ├── columns: a:18 b:19 c:20 d:21 - │ │ └── mapping: - │ │ ├── k:6 => a:18 - │ │ ├── v:7 => b:19 - │ │ ├── x:9 => c:20 - │ │ └── y:10 => d:21 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:13!null uniq_overlaps_pk.b:14!null uniq_overlaps_pk.c:15 uniq_overlaps_pk.d:16 - │ └── filters - │ ├── b:19 = uniq_overlaps_pk.b:14 - │ ├── c:20 = uniq_overlaps_pk.c:15 - │ └── a:18 != uniq_overlaps_pk.a:13 + │ └── project + │ ├── columns: b:19 c:20 + │ └── semi-join (hash) + │ ├── columns: a:18 b:19 c:20 d:21 + │ ├── with-scan &1 + │ │ ├── columns: a:18 b:19 c:20 d:21 + │ │ └── mapping: + │ │ ├── k:6 => a:18 + │ │ ├── v:7 => b:19 + │ │ ├── x:9 => c:20 + │ │ └── y:10 => d:21 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:13!null uniq_overlaps_pk.b:14!null uniq_overlaps_pk.c:15 uniq_overlaps_pk.d:16 + │ └── filters + │ ├── b:19 = uniq_overlaps_pk.b:14 + │ ├── c:20 = uniq_overlaps_pk.c:15 + │ └── a:18 != uniq_overlaps_pk.a:13 ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:27 b:28 c:29 d:30 - │ ├── with-scan &1 - │ │ ├── columns: a:27 b:28 c:29 d:30 - │ │ └── mapping: - │ │ ├── k:6 => a:27 - │ │ ├── v:7 => b:28 - │ │ ├── x:9 => c:29 - │ │ └── y:10 => d:30 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:22!null uniq_overlaps_pk.b:23!null uniq_overlaps_pk.c:24 uniq_overlaps_pk.d:25 - │ └── filters - │ ├── a:27 = uniq_overlaps_pk.a:22 - │ └── b:28 != uniq_overlaps_pk.b:23 + │ └── project + │ ├── columns: a:27 + │ └── semi-join (hash) + │ ├── columns: a:27 b:28 c:29 d:30 + │ ├── with-scan &1 + │ │ ├── columns: a:27 b:28 c:29 d:30 + │ │ └── mapping: + │ │ ├── k:6 => a:27 + │ │ ├── v:7 => b:28 + │ │ ├── x:9 => c:29 + │ │ └── y:10 => d:30 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:22!null uniq_overlaps_pk.b:23!null uniq_overlaps_pk.c:24 uniq_overlaps_pk.d:25 + │ └── filters + │ ├── a:27 = uniq_overlaps_pk.a:22 + │ └── b:28 != uniq_overlaps_pk.b:23 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:36 b:37 c:38 d:39 - ├── with-scan &1 - │ ├── columns: a:36 b:37 c:38 d:39 - │ └── mapping: - │ ├── k:6 => a:36 - │ ├── v:7 => b:37 - │ ├── x:9 => c:38 - │ └── y:10 => d:39 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:31!null uniq_overlaps_pk.b:32!null uniq_overlaps_pk.c:33 uniq_overlaps_pk.d:34 - └── filters - ├── c:38 = uniq_overlaps_pk.c:33 - ├── d:39 = uniq_overlaps_pk.d:34 - └── (a:36 != uniq_overlaps_pk.a:31) OR (b:37 != uniq_overlaps_pk.b:32) + └── project + ├── columns: c:38 d:39 + └── semi-join (hash) + ├── columns: a:36 b:37 c:38 d:39 + ├── with-scan &1 + │ ├── columns: a:36 b:37 c:38 d:39 + │ └── mapping: + │ ├── k:6 => a:36 + │ ├── v:7 => b:37 + │ ├── x:9 => c:38 + │ └── y:10 => d:39 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:31!null uniq_overlaps_pk.b:32!null uniq_overlaps_pk.c:33 uniq_overlaps_pk.d:34 + └── filters + ├── c:38 = uniq_overlaps_pk.c:33 + ├── d:39 = uniq_overlaps_pk.d:34 + └── (a:36 != uniq_overlaps_pk.a:31) OR (b:37 != uniq_overlaps_pk.b:32) exec-ddl CREATE TABLE uniq_hidden_pk ( @@ -621,56 +653,62 @@ insert uniq_hidden_pk │ └── unique_rowid() [as=column11:11] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:18!null b:19!null c:20!null d:21!null rowid:22 - │ ├── with-scan &1 - │ │ ├── columns: a:18!null b:19!null c:20!null d:21!null rowid:22 - │ │ └── mapping: - │ │ ├── column1:7 => a:18 - │ │ ├── column2:8 => b:19 - │ │ ├── column3:9 => c:20 - │ │ ├── column4:10 => d:21 - │ │ └── column11:11 => rowid:22 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:12 uniq_hidden_pk.b:13 uniq_hidden_pk.c:14 uniq_hidden_pk.d:15 uniq_hidden_pk.rowid:16!null - │ └── filters - │ ├── b:19 = uniq_hidden_pk.b:13 - │ ├── c:20 = uniq_hidden_pk.c:14 - │ └── rowid:22 != uniq_hidden_pk.rowid:16 + │ └── project + │ ├── columns: b:19!null c:20!null + │ └── semi-join (hash) + │ ├── columns: a:18!null b:19!null c:20!null d:21!null rowid:22 + │ ├── with-scan &1 + │ │ ├── columns: a:18!null b:19!null c:20!null d:21!null rowid:22 + │ │ └── mapping: + │ │ ├── column1:7 => a:18 + │ │ ├── column2:8 => b:19 + │ │ ├── column3:9 => c:20 + │ │ ├── column4:10 => d:21 + │ │ └── column11:11 => rowid:22 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:12 uniq_hidden_pk.b:13 uniq_hidden_pk.c:14 uniq_hidden_pk.d:15 uniq_hidden_pk.rowid:16!null + │ └── filters + │ ├── b:19 = uniq_hidden_pk.b:13 + │ ├── c:20 = uniq_hidden_pk.c:14 + │ └── rowid:22 != uniq_hidden_pk.rowid:16 ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:29!null b:30!null c:31!null d:32!null rowid:33 - │ ├── with-scan &1 - │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null rowid:33 - │ │ └── mapping: - │ │ ├── column1:7 => a:29 - │ │ ├── column2:8 => b:30 - │ │ ├── column3:9 => c:31 - │ │ ├── column4:10 => d:32 - │ │ └── column11:11 => rowid:33 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:23 uniq_hidden_pk.b:24 uniq_hidden_pk.c:25 uniq_hidden_pk.d:26 uniq_hidden_pk.rowid:27!null - │ └── filters - │ ├── a:29 = uniq_hidden_pk.a:23 - │ ├── b:30 = uniq_hidden_pk.b:24 - │ ├── d:32 = uniq_hidden_pk.d:26 - │ └── rowid:33 != uniq_hidden_pk.rowid:27 + │ └── project + │ ├── columns: a:29!null b:30!null d:32!null + │ └── semi-join (hash) + │ ├── columns: a:29!null b:30!null c:31!null d:32!null rowid:33 + │ ├── with-scan &1 + │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null rowid:33 + │ │ └── mapping: + │ │ ├── column1:7 => a:29 + │ │ ├── column2:8 => b:30 + │ │ ├── column3:9 => c:31 + │ │ ├── column4:10 => d:32 + │ │ └── column11:11 => rowid:33 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:23 uniq_hidden_pk.b:24 uniq_hidden_pk.c:25 uniq_hidden_pk.d:26 uniq_hidden_pk.rowid:27!null + │ └── filters + │ ├── a:29 = uniq_hidden_pk.a:23 + │ ├── b:30 = uniq_hidden_pk.b:24 + │ ├── d:32 = uniq_hidden_pk.d:26 + │ └── rowid:33 != uniq_hidden_pk.rowid:27 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:40!null b:41!null c:42!null d:43!null rowid:44 - ├── with-scan &1 - │ ├── columns: a:40!null b:41!null c:42!null d:43!null rowid:44 - │ └── mapping: - │ ├── column1:7 => a:40 - │ ├── column2:8 => b:41 - │ ├── column3:9 => c:42 - │ ├── column4:10 => d:43 - │ └── column11:11 => rowid:44 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:34 uniq_hidden_pk.b:35 uniq_hidden_pk.c:36 uniq_hidden_pk.d:37 uniq_hidden_pk.rowid:38!null - └── filters - ├── a:40 = uniq_hidden_pk.a:34 - └── rowid:44 != uniq_hidden_pk.rowid:38 + └── project + ├── columns: a:40!null + └── semi-join (hash) + ├── columns: a:40!null b:41!null c:42!null d:43!null rowid:44 + ├── with-scan &1 + │ ├── columns: a:40!null b:41!null c:42!null d:43!null rowid:44 + │ └── mapping: + │ ├── column1:7 => a:40 + │ ├── column2:8 => b:41 + │ ├── column3:9 => c:42 + │ ├── column4:10 => d:43 + │ └── column11:11 => rowid:44 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:34 uniq_hidden_pk.b:35 uniq_hidden_pk.c:36 uniq_hidden_pk.d:37 uniq_hidden_pk.rowid:38!null + └── filters + ├── a:40 = uniq_hidden_pk.a:34 + └── rowid:44 != uniq_hidden_pk.rowid:38 # Insert with non-constant input. # Add inequality filters for the hidden primary key column. @@ -696,56 +734,62 @@ insert uniq_hidden_pk │ └── unique_rowid() [as=column14:14] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 - │ ├── with-scan &1 - │ │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 - │ │ └── mapping: - │ │ ├── k:7 => a:21 - │ │ ├── v:8 => b:22 - │ │ ├── x:10 => c:23 - │ │ ├── y:11 => d:24 - │ │ └── column14:14 => rowid:25 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:15 uniq_hidden_pk.b:16 uniq_hidden_pk.c:17 uniq_hidden_pk.d:18 uniq_hidden_pk.rowid:19!null - │ └── filters - │ ├── b:22 = uniq_hidden_pk.b:16 - │ ├── c:23 = uniq_hidden_pk.c:17 - │ └── rowid:25 != uniq_hidden_pk.rowid:19 + │ └── project + │ ├── columns: b:22 c:23 + │ └── semi-join (hash) + │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 + │ ├── with-scan &1 + │ │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 + │ │ └── mapping: + │ │ ├── k:7 => a:21 + │ │ ├── v:8 => b:22 + │ │ ├── x:10 => c:23 + │ │ ├── y:11 => d:24 + │ │ └── column14:14 => rowid:25 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:15 uniq_hidden_pk.b:16 uniq_hidden_pk.c:17 uniq_hidden_pk.d:18 uniq_hidden_pk.rowid:19!null + │ └── filters + │ ├── b:22 = uniq_hidden_pk.b:16 + │ ├── c:23 = uniq_hidden_pk.c:17 + │ └── rowid:25 != uniq_hidden_pk.rowid:19 ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 - │ ├── with-scan &1 - │ │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 - │ │ └── mapping: - │ │ ├── k:7 => a:32 - │ │ ├── v:8 => b:33 - │ │ ├── x:10 => c:34 - │ │ ├── y:11 => d:35 - │ │ └── column14:14 => rowid:36 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:26 uniq_hidden_pk.b:27 uniq_hidden_pk.c:28 uniq_hidden_pk.d:29 uniq_hidden_pk.rowid:30!null - │ └── filters - │ ├── a:32 = uniq_hidden_pk.a:26 - │ ├── b:33 = uniq_hidden_pk.b:27 - │ ├── d:35 = uniq_hidden_pk.d:29 - │ └── rowid:36 != uniq_hidden_pk.rowid:30 + │ └── project + │ ├── columns: a:32 b:33 d:35 + │ └── semi-join (hash) + │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 + │ ├── with-scan &1 + │ │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 + │ │ └── mapping: + │ │ ├── k:7 => a:32 + │ │ ├── v:8 => b:33 + │ │ ├── x:10 => c:34 + │ │ ├── y:11 => d:35 + │ │ └── column14:14 => rowid:36 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:26 uniq_hidden_pk.b:27 uniq_hidden_pk.c:28 uniq_hidden_pk.d:29 uniq_hidden_pk.rowid:30!null + │ └── filters + │ ├── a:32 = uniq_hidden_pk.a:26 + │ ├── b:33 = uniq_hidden_pk.b:27 + │ ├── d:35 = uniq_hidden_pk.d:29 + │ └── rowid:36 != uniq_hidden_pk.rowid:30 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:43 b:44 c:45 d:46 rowid:47 - ├── with-scan &1 - │ ├── columns: a:43 b:44 c:45 d:46 rowid:47 - │ └── mapping: - │ ├── k:7 => a:43 - │ ├── v:8 => b:44 - │ ├── x:10 => c:45 - │ ├── y:11 => d:46 - │ └── column14:14 => rowid:47 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:37 uniq_hidden_pk.b:38 uniq_hidden_pk.c:39 uniq_hidden_pk.d:40 uniq_hidden_pk.rowid:41!null - └── filters - ├── a:43 = uniq_hidden_pk.a:37 - └── rowid:47 != uniq_hidden_pk.rowid:41 + └── project + ├── columns: a:43 + └── semi-join (hash) + ├── columns: a:43 b:44 c:45 d:46 rowid:47 + ├── with-scan &1 + │ ├── columns: a:43 b:44 c:45 d:46 rowid:47 + │ └── mapping: + │ ├── k:7 => a:43 + │ ├── v:8 => b:44 + │ ├── x:10 => c:45 + │ ├── y:11 => d:46 + │ └── column14:14 => rowid:47 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:37 uniq_hidden_pk.b:38 uniq_hidden_pk.c:39 uniq_hidden_pk.d:40 uniq_hidden_pk.rowid:41!null + └── filters + ├── a:43 = uniq_hidden_pk.a:37 + └── rowid:47 != uniq_hidden_pk.rowid:41 exec-ddl CREATE TABLE uniq_partial ( @@ -773,21 +817,23 @@ insert uniq_partial │ └── (2, 2, 2) └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:12!null a:13!null b:14!null - ├── with-scan &1 - │ ├── columns: k:12!null a:13!null b:14!null - │ └── mapping: - │ ├── column1:5 => k:12 - │ ├── column2:6 => a:13 - │ └── column3:7 => b:14 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 - └── filters - ├── a:13 = uniq_partial.a:9 - ├── b:14 > 0 - ├── uniq_partial.b:10 > 0 - └── k:12 != uniq_partial.k:8 + └── project + ├── columns: a:13!null + └── semi-join (hash) + ├── columns: k:12!null a:13!null b:14!null + ├── with-scan &1 + │ ├── columns: k:12!null a:13!null b:14!null + │ └── mapping: + │ ├── column1:5 => k:12 + │ ├── column2:6 => a:13 + │ └── column3:7 => b:14 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 + └── filters + ├── a:13 = uniq_partial.a:9 + ├── b:14 > 0 + ├── uniq_partial.b:10 > 0 + └── k:12 != uniq_partial.k:8 # Some of the inserted values have nulls. build @@ -807,21 +853,23 @@ insert uniq_partial │ └── (3, NULL::INT8, 3) └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:12!null a:13 b:14!null - ├── with-scan &1 - │ ├── columns: k:12!null a:13 b:14!null - │ └── mapping: - │ ├── column1:5 => k:12 - │ ├── column2:6 => a:13 - │ └── column3:7 => b:14 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 - └── filters - ├── a:13 = uniq_partial.a:9 - ├── b:14 > 0 - ├── uniq_partial.b:10 > 0 - └── k:12 != uniq_partial.k:8 + └── project + ├── columns: a:13 + └── semi-join (hash) + ├── columns: k:12!null a:13 b:14!null + ├── with-scan &1 + │ ├── columns: k:12!null a:13 b:14!null + │ └── mapping: + │ ├── column1:5 => k:12 + │ ├── column2:6 => a:13 + │ └── column3:7 => b:14 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 + └── filters + ├── a:13 = uniq_partial.a:9 + ├── b:14 > 0 + ├── uniq_partial.b:10 > 0 + └── k:12 != uniq_partial.k:8 # No need to plan checks for a since it's always null. build @@ -959,21 +1007,23 @@ insert uniq_partial │ └── columns: other.k:5 v:6 w:7!null x:8 y:9 rowid:10!null other.crdb_internal_mvcc_timestamp:11 └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:16 a:17 b:18!null - ├── with-scan &1 - │ ├── columns: k:16 a:17 b:18!null - │ └── mapping: - │ ├── other.k:5 => k:16 - │ ├── v:6 => a:17 - │ └── w:7 => b:18 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 - └── filters - ├── a:17 = uniq_partial.a:13 - ├── b:18 > 0 - ├── uniq_partial.b:14 > 0 - └── k:16 != uniq_partial.k:12 + └── project + ├── columns: a:17 + └── semi-join (hash) + ├── columns: k:16 a:17 b:18!null + ├── with-scan &1 + │ ├── columns: k:16 a:17 b:18!null + │ └── mapping: + │ ├── other.k:5 => k:16 + │ ├── v:6 => a:17 + │ └── w:7 => b:18 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 + └── filters + ├── a:17 = uniq_partial.a:13 + ├── b:18 > 0 + ├── uniq_partial.b:14 > 0 + └── k:16 != uniq_partial.k:12 exec-ddl CREATE TABLE uniq_partial_overlaps_pk ( @@ -1010,57 +1060,63 @@ insert uniq_partial_overlaps_pk │ └── (2, 2, 2, 2) └── unique-checks ├── unique-checks-item: uniq_partial_overlaps_pk(c) - │ └── semi-join (hash) - │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ ├── with-scan &1 - │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ │ └── mapping: - │ │ ├── column1:6 => a:15 - │ │ ├── column2:7 => b:16 - │ │ ├── column3:8 => c:17 - │ │ └── column4:9 => d:18 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:10!null uniq_partial_overlaps_pk.b:11!null uniq_partial_overlaps_pk.c:12 uniq_partial_overlaps_pk.d:13 - │ └── filters - │ ├── c:17 = uniq_partial_overlaps_pk.c:12 - │ ├── d:18 > 0 - │ ├── uniq_partial_overlaps_pk.d:13 > 0 - │ └── (a:15 != uniq_partial_overlaps_pk.a:10) OR (b:16 != uniq_partial_overlaps_pk.b:11) + │ └── project + │ ├── columns: c:17!null + │ └── semi-join (hash) + │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ ├── with-scan &1 + │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ │ └── mapping: + │ │ ├── column1:6 => a:15 + │ │ ├── column2:7 => b:16 + │ │ ├── column3:8 => c:17 + │ │ └── column4:9 => d:18 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:10!null uniq_partial_overlaps_pk.b:11!null uniq_partial_overlaps_pk.c:12 uniq_partial_overlaps_pk.d:13 + │ └── filters + │ ├── c:17 = uniq_partial_overlaps_pk.c:12 + │ ├── d:18 > 0 + │ ├── uniq_partial_overlaps_pk.d:13 > 0 + │ └── (a:15 != uniq_partial_overlaps_pk.a:10) OR (b:16 != uniq_partial_overlaps_pk.b:11) ├── unique-checks-item: uniq_partial_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ ├── with-scan &1 - │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ │ └── mapping: - │ │ ├── column1:6 => a:24 - │ │ ├── column2:7 => b:25 - │ │ ├── column3:8 => c:26 - │ │ └── column4:9 => d:27 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:19!null uniq_partial_overlaps_pk.b:20!null uniq_partial_overlaps_pk.c:21 uniq_partial_overlaps_pk.d:22 - │ └── filters - │ ├── a:24 = uniq_partial_overlaps_pk.a:19 - │ ├── d:27 > 0 - │ ├── uniq_partial_overlaps_pk.d:22 > 0 - │ └── b:25 != uniq_partial_overlaps_pk.b:20 + │ └── project + │ ├── columns: a:24!null + │ └── semi-join (hash) + │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ ├── with-scan &1 + │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ │ └── mapping: + │ │ ├── column1:6 => a:24 + │ │ ├── column2:7 => b:25 + │ │ ├── column3:8 => c:26 + │ │ └── column4:9 => d:27 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:19!null uniq_partial_overlaps_pk.b:20!null uniq_partial_overlaps_pk.c:21 uniq_partial_overlaps_pk.d:22 + │ └── filters + │ ├── a:24 = uniq_partial_overlaps_pk.a:19 + │ ├── d:27 > 0 + │ ├── uniq_partial_overlaps_pk.d:22 > 0 + │ └── b:25 != uniq_partial_overlaps_pk.b:20 └── unique-checks-item: uniq_partial_overlaps_pk(b,c) - └── semi-join (hash) - ├── columns: a:33!null b:34!null c:35!null d:36!null - ├── with-scan &1 - │ ├── columns: a:33!null b:34!null c:35!null d:36!null - │ └── mapping: - │ ├── column1:6 => a:33 - │ ├── column2:7 => b:34 - │ ├── column3:8 => c:35 - │ └── column4:9 => d:36 - ├── scan uniq_partial_overlaps_pk - │ └── columns: uniq_partial_overlaps_pk.a:28!null uniq_partial_overlaps_pk.b:29!null uniq_partial_overlaps_pk.c:30 uniq_partial_overlaps_pk.d:31 - └── filters - ├── b:34 = uniq_partial_overlaps_pk.b:29 - ├── c:35 = uniq_partial_overlaps_pk.c:30 - ├── d:36 > 0 - ├── uniq_partial_overlaps_pk.d:31 > 0 - └── a:33 != uniq_partial_overlaps_pk.a:28 + └── project + ├── columns: b:34!null c:35!null + └── semi-join (hash) + ├── columns: a:33!null b:34!null c:35!null d:36!null + ├── with-scan &1 + │ ├── columns: a:33!null b:34!null c:35!null d:36!null + │ └── mapping: + │ ├── column1:6 => a:33 + │ ├── column2:7 => b:34 + │ ├── column3:8 => c:35 + │ └── column4:9 => d:36 + ├── scan uniq_partial_overlaps_pk + │ └── columns: uniq_partial_overlaps_pk.a:28!null uniq_partial_overlaps_pk.b:29!null uniq_partial_overlaps_pk.c:30 uniq_partial_overlaps_pk.d:31 + └── filters + ├── b:34 = uniq_partial_overlaps_pk.b:29 + ├── c:35 = uniq_partial_overlaps_pk.c:30 + ├── d:36 > 0 + ├── uniq_partial_overlaps_pk.d:31 > 0 + └── a:33 != uniq_partial_overlaps_pk.a:28 # Insert with non-constant input. # Do not build uniqueness checks when the primary key columns are a subset of @@ -1082,57 +1138,63 @@ insert uniq_partial_overlaps_pk │ └── columns: k:6 v:7 w:8!null x:9 y:10 rowid:11!null other.crdb_internal_mvcc_timestamp:12 └── unique-checks ├── unique-checks-item: uniq_partial_overlaps_pk(c) - │ └── semi-join (hash) - │ ├── columns: a:18 b:19 c:20 d:21 - │ ├── with-scan &1 - │ │ ├── columns: a:18 b:19 c:20 d:21 - │ │ └── mapping: - │ │ ├── k:6 => a:18 - │ │ ├── v:7 => b:19 - │ │ ├── x:9 => c:20 - │ │ └── y:10 => d:21 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:13!null uniq_partial_overlaps_pk.b:14!null uniq_partial_overlaps_pk.c:15 uniq_partial_overlaps_pk.d:16 - │ └── filters - │ ├── c:20 = uniq_partial_overlaps_pk.c:15 - │ ├── d:21 > 0 - │ ├── uniq_partial_overlaps_pk.d:16 > 0 - │ └── (a:18 != uniq_partial_overlaps_pk.a:13) OR (b:19 != uniq_partial_overlaps_pk.b:14) + │ └── project + │ ├── columns: c:20 + │ └── semi-join (hash) + │ ├── columns: a:18 b:19 c:20 d:21 + │ ├── with-scan &1 + │ │ ├── columns: a:18 b:19 c:20 d:21 + │ │ └── mapping: + │ │ ├── k:6 => a:18 + │ │ ├── v:7 => b:19 + │ │ ├── x:9 => c:20 + │ │ └── y:10 => d:21 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:13!null uniq_partial_overlaps_pk.b:14!null uniq_partial_overlaps_pk.c:15 uniq_partial_overlaps_pk.d:16 + │ └── filters + │ ├── c:20 = uniq_partial_overlaps_pk.c:15 + │ ├── d:21 > 0 + │ ├── uniq_partial_overlaps_pk.d:16 > 0 + │ └── (a:18 != uniq_partial_overlaps_pk.a:13) OR (b:19 != uniq_partial_overlaps_pk.b:14) ├── unique-checks-item: uniq_partial_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:27 b:28 c:29 d:30 - │ ├── with-scan &1 - │ │ ├── columns: a:27 b:28 c:29 d:30 - │ │ └── mapping: - │ │ ├── k:6 => a:27 - │ │ ├── v:7 => b:28 - │ │ ├── x:9 => c:29 - │ │ └── y:10 => d:30 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:22!null uniq_partial_overlaps_pk.b:23!null uniq_partial_overlaps_pk.c:24 uniq_partial_overlaps_pk.d:25 - │ └── filters - │ ├── a:27 = uniq_partial_overlaps_pk.a:22 - │ ├── d:30 > 0 - │ ├── uniq_partial_overlaps_pk.d:25 > 0 - │ └── b:28 != uniq_partial_overlaps_pk.b:23 + │ └── project + │ ├── columns: a:27 + │ └── semi-join (hash) + │ ├── columns: a:27 b:28 c:29 d:30 + │ ├── with-scan &1 + │ │ ├── columns: a:27 b:28 c:29 d:30 + │ │ └── mapping: + │ │ ├── k:6 => a:27 + │ │ ├── v:7 => b:28 + │ │ ├── x:9 => c:29 + │ │ └── y:10 => d:30 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:22!null uniq_partial_overlaps_pk.b:23!null uniq_partial_overlaps_pk.c:24 uniq_partial_overlaps_pk.d:25 + │ └── filters + │ ├── a:27 = uniq_partial_overlaps_pk.a:22 + │ ├── d:30 > 0 + │ ├── uniq_partial_overlaps_pk.d:25 > 0 + │ └── b:28 != uniq_partial_overlaps_pk.b:23 └── unique-checks-item: uniq_partial_overlaps_pk(b,c) - └── semi-join (hash) - ├── columns: a:36 b:37 c:38 d:39 - ├── with-scan &1 - │ ├── columns: a:36 b:37 c:38 d:39 - │ └── mapping: - │ ├── k:6 => a:36 - │ ├── v:7 => b:37 - │ ├── x:9 => c:38 - │ └── y:10 => d:39 - ├── scan uniq_partial_overlaps_pk - │ └── columns: uniq_partial_overlaps_pk.a:31!null uniq_partial_overlaps_pk.b:32!null uniq_partial_overlaps_pk.c:33 uniq_partial_overlaps_pk.d:34 - └── filters - ├── b:37 = uniq_partial_overlaps_pk.b:32 - ├── c:38 = uniq_partial_overlaps_pk.c:33 - ├── d:39 > 0 - ├── uniq_partial_overlaps_pk.d:34 > 0 - └── a:36 != uniq_partial_overlaps_pk.a:31 + └── project + ├── columns: b:37 c:38 + └── semi-join (hash) + ├── columns: a:36 b:37 c:38 d:39 + ├── with-scan &1 + │ ├── columns: a:36 b:37 c:38 d:39 + │ └── mapping: + │ ├── k:6 => a:36 + │ ├── v:7 => b:37 + │ ├── x:9 => c:38 + │ └── y:10 => d:39 + ├── scan uniq_partial_overlaps_pk + │ └── columns: uniq_partial_overlaps_pk.a:31!null uniq_partial_overlaps_pk.b:32!null uniq_partial_overlaps_pk.c:33 uniq_partial_overlaps_pk.d:34 + └── filters + ├── b:37 = uniq_partial_overlaps_pk.b:32 + ├── c:38 = uniq_partial_overlaps_pk.c:33 + ├── d:39 > 0 + ├── uniq_partial_overlaps_pk.d:34 > 0 + └── a:36 != uniq_partial_overlaps_pk.a:31 exec-ddl CREATE TABLE uniq_partial_hidden_pk ( @@ -1167,22 +1229,24 @@ insert uniq_partial_hidden_pk │ └── unique_rowid() [as=column9:9] └── unique-checks └── unique-checks-item: uniq_partial_hidden_pk(b) - └── semi-join (hash) - ├── columns: a:15!null b:16!null c:17 rowid:18 - ├── with-scan &1 - │ ├── columns: a:15!null b:16!null c:17 rowid:18 - │ └── mapping: - │ ├── column1:6 => a:15 - │ ├── column2:7 => b:16 - │ ├── column8:8 => c:17 - │ └── column9:9 => rowid:18 - ├── scan uniq_partial_hidden_pk - │ └── columns: uniq_partial_hidden_pk.a:10 uniq_partial_hidden_pk.b:11 uniq_partial_hidden_pk.c:12 uniq_partial_hidden_pk.rowid:13!null - └── filters - ├── b:16 = uniq_partial_hidden_pk.b:11 - ├── c:17 > 0 - ├── uniq_partial_hidden_pk.c:12 > 0 - └── rowid:18 != uniq_partial_hidden_pk.rowid:13 + └── project + ├── columns: b:16!null + └── semi-join (hash) + ├── columns: a:15!null b:16!null c:17 rowid:18 + ├── with-scan &1 + │ ├── columns: a:15!null b:16!null c:17 rowid:18 + │ └── mapping: + │ ├── column1:6 => a:15 + │ ├── column2:7 => b:16 + │ ├── column8:8 => c:17 + │ └── column9:9 => rowid:18 + ├── scan uniq_partial_hidden_pk + │ └── columns: uniq_partial_hidden_pk.a:10 uniq_partial_hidden_pk.b:11 uniq_partial_hidden_pk.c:12 uniq_partial_hidden_pk.rowid:13!null + └── filters + ├── b:16 = uniq_partial_hidden_pk.b:11 + ├── c:17 > 0 + ├── uniq_partial_hidden_pk.c:12 > 0 + └── rowid:18 != uniq_partial_hidden_pk.rowid:13 # Add inequality filters for the hidden primary key column. build @@ -1207,22 +1271,24 @@ insert uniq_partial_hidden_pk │ └── unique_rowid() [as=column14:14] └── unique-checks └── unique-checks-item: uniq_partial_hidden_pk(b) - └── semi-join (hash) - ├── columns: a:20 b:21 c:22 rowid:23 - ├── with-scan &1 - │ ├── columns: a:20 b:21 c:22 rowid:23 - │ └── mapping: - │ ├── k:6 => a:20 - │ ├── v:7 => b:21 - │ ├── column13:13 => c:22 - │ └── column14:14 => rowid:23 - ├── scan uniq_partial_hidden_pk - │ └── columns: uniq_partial_hidden_pk.a:15 uniq_partial_hidden_pk.b:16 uniq_partial_hidden_pk.c:17 uniq_partial_hidden_pk.rowid:18!null - └── filters - ├── b:21 = uniq_partial_hidden_pk.b:16 - ├── c:22 > 0 - ├── uniq_partial_hidden_pk.c:17 > 0 - └── rowid:23 != uniq_partial_hidden_pk.rowid:18 + └── project + ├── columns: b:21 + └── semi-join (hash) + ├── columns: a:20 b:21 c:22 rowid:23 + ├── with-scan &1 + │ ├── columns: a:20 b:21 c:22 rowid:23 + │ └── mapping: + │ ├── k:6 => a:20 + │ ├── v:7 => b:21 + │ ├── column13:13 => c:22 + │ └── column14:14 => rowid:23 + ├── scan uniq_partial_hidden_pk + │ └── columns: uniq_partial_hidden_pk.a:15 uniq_partial_hidden_pk.b:16 uniq_partial_hidden_pk.c:17 uniq_partial_hidden_pk.rowid:18!null + └── filters + ├── b:21 = uniq_partial_hidden_pk.b:16 + ├── c:22 > 0 + ├── uniq_partial_hidden_pk.c:17 > 0 + └── rowid:23 != uniq_partial_hidden_pk.rowid:18 exec-ddl CREATE TABLE uniq_partial_constraint_and_index ( @@ -1270,29 +1336,31 @@ insert uniq_partial_constraint_and_index │ └── true [as=partial_index_put1:13] └── unique-checks └── unique-checks-item: uniq_partial_constraint_and_index(a) - └── semi-join (hash) - ├── columns: k:18!null a:19!null b:20!null - ├── select - │ ├── columns: k:18!null a:19!null b:20!null - │ ├── with-scan &1 - │ │ ├── columns: k:18!null a:19!null b:20!null - │ │ └── mapping: - │ │ ├── column1:5 => k:18 - │ │ ├── column2:6 => a:19 - │ │ └── column3:7 => b:20 - │ └── filters - │ └── b:20 > 10 - ├── select - │ ├── columns: uniq_partial_constraint_and_index.k:14!null uniq_partial_constraint_and_index.a:15 uniq_partial_constraint_and_index.b:16!null - │ ├── scan uniq_partial_constraint_and_index - │ │ ├── columns: uniq_partial_constraint_and_index.k:14!null uniq_partial_constraint_and_index.a:15 uniq_partial_constraint_and_index.b:16 - │ │ └── partial index predicates - │ │ └── secondary: filters (true) - │ └── filters - │ └── uniq_partial_constraint_and_index.b:16 > 10 - └── filters - ├── a:19 = uniq_partial_constraint_and_index.a:15 - └── k:18 != uniq_partial_constraint_and_index.k:14 + └── project + ├── columns: a:19!null + └── semi-join (hash) + ├── columns: k:18!null a:19!null b:20!null + ├── select + │ ├── columns: k:18!null a:19!null b:20!null + │ ├── with-scan &1 + │ │ ├── columns: k:18!null a:19!null b:20!null + │ │ └── mapping: + │ │ ├── column1:5 => k:18 + │ │ ├── column2:6 => a:19 + │ │ └── column3:7 => b:20 + │ └── filters + │ └── b:20 > 10 + ├── select + │ ├── columns: uniq_partial_constraint_and_index.k:14!null uniq_partial_constraint_and_index.a:15 uniq_partial_constraint_and_index.b:16!null + │ ├── scan uniq_partial_constraint_and_index + │ │ ├── columns: uniq_partial_constraint_and_index.k:14!null uniq_partial_constraint_and_index.a:15 uniq_partial_constraint_and_index.b:16 + │ │ └── partial index predicates + │ │ └── secondary: filters (true) + │ └── filters + │ └── uniq_partial_constraint_and_index.b:16 > 10 + └── filters + ├── a:19 = uniq_partial_constraint_and_index.a:15 + └── k:18 != uniq_partial_constraint_and_index.k:14 exec-ddl CREATE TABLE uniq_constraint_and_partial_index ( @@ -1474,33 +1542,35 @@ insert uniq_computed_pk │ └── column3:11::STRING [as=column13:13] └── unique-checks └── unique-checks-item: uniq_computed_pk(d) - └── semi-join (hash) - ├── columns: i:38!null s:39!null d:40!null c_i_expr:41!null c_s:42!null c_d:43!null c_d_expr:44!null - ├── with-scan &1 - │ ├── columns: i:38!null s:39!null d:40!null c_i_expr:41!null c_s:42!null c_d:43!null c_d_expr:44!null - │ └── mapping: - │ ├── column1:9 => i:38 - │ ├── column2:10 => s:39 - │ ├── column3:11 => d:40 - │ ├── column12:12 => c_i_expr:41 - │ ├── column2:10 => c_s:42 - │ ├── column3:11 => c_d:43 - │ └── column13:13 => c_d_expr:44 - ├── project - │ ├── columns: uniq_computed_pk.c_s:34 uniq_computed_pk.i:30!null uniq_computed_pk.s:31 uniq_computed_pk.d:32 uniq_computed_pk.c_i_expr:33!null uniq_computed_pk.c_d:35 uniq_computed_pk.c_d_expr:36 - │ ├── scan uniq_computed_pk - │ │ ├── columns: uniq_computed_pk.i:30!null uniq_computed_pk.s:31 uniq_computed_pk.d:32 uniq_computed_pk.c_i_expr:33!null uniq_computed_pk.c_d:35 uniq_computed_pk.c_d_expr:36 - │ │ └── computed column expressions - │ │ ├── uniq_computed_pk.c_i_expr:33 - │ │ │ └── CASE WHEN uniq_computed_pk.i:30 < 0 THEN 'foo' ELSE 'bar' END - │ │ ├── uniq_computed_pk.c_s:34 - │ │ │ └── uniq_computed_pk.s:31 - │ │ ├── uniq_computed_pk.c_d:35 - │ │ │ └── uniq_computed_pk.d:32 - │ │ └── uniq_computed_pk.c_d_expr:36 - │ │ └── uniq_computed_pk.d:32::STRING - │ └── projections - │ └── uniq_computed_pk.s:31 [as=uniq_computed_pk.c_s:34] - └── filters - ├── d:40 = uniq_computed_pk.d:32 - └── (i:38 != uniq_computed_pk.i:30) OR (c_i_expr:41 != uniq_computed_pk.c_i_expr:33) + └── project + ├── columns: d:40!null + └── semi-join (hash) + ├── columns: i:38!null s:39!null d:40!null c_i_expr:41!null c_s:42!null c_d:43!null c_d_expr:44!null + ├── with-scan &1 + │ ├── columns: i:38!null s:39!null d:40!null c_i_expr:41!null c_s:42!null c_d:43!null c_d_expr:44!null + │ └── mapping: + │ ├── column1:9 => i:38 + │ ├── column2:10 => s:39 + │ ├── column3:11 => d:40 + │ ├── column12:12 => c_i_expr:41 + │ ├── column2:10 => c_s:42 + │ ├── column3:11 => c_d:43 + │ └── column13:13 => c_d_expr:44 + ├── project + │ ├── columns: uniq_computed_pk.c_s:34 uniq_computed_pk.i:30!null uniq_computed_pk.s:31 uniq_computed_pk.d:32 uniq_computed_pk.c_i_expr:33!null uniq_computed_pk.c_d:35 uniq_computed_pk.c_d_expr:36 + │ ├── scan uniq_computed_pk + │ │ ├── columns: uniq_computed_pk.i:30!null uniq_computed_pk.s:31 uniq_computed_pk.d:32 uniq_computed_pk.c_i_expr:33!null uniq_computed_pk.c_d:35 uniq_computed_pk.c_d_expr:36 + │ │ └── computed column expressions + │ │ ├── uniq_computed_pk.c_i_expr:33 + │ │ │ └── CASE WHEN uniq_computed_pk.i:30 < 0 THEN 'foo' ELSE 'bar' END + │ │ ├── uniq_computed_pk.c_s:34 + │ │ │ └── uniq_computed_pk.s:31 + │ │ ├── uniq_computed_pk.c_d:35 + │ │ │ └── uniq_computed_pk.d:32 + │ │ └── uniq_computed_pk.c_d_expr:36 + │ │ └── uniq_computed_pk.d:32::STRING + │ └── projections + │ └── uniq_computed_pk.s:31 [as=uniq_computed_pk.c_s:34] + └── filters + ├── d:40 = uniq_computed_pk.d:32 + └── (i:38 != uniq_computed_pk.i:30) OR (c_i_expr:41 != uniq_computed_pk.c_i_expr:33) diff --git a/pkg/sql/opt/optbuilder/testdata/unique-checks-update b/pkg/sql/opt/optbuilder/testdata/unique-checks-update index 8a3398c8ec03..bc32d4924c1a 100644 --- a/pkg/sql/opt/optbuilder/testdata/unique-checks-update +++ b/pkg/sql/opt/optbuilder/testdata/unique-checks-update @@ -29,38 +29,42 @@ update uniq │ └── 2 [as=x_new:14] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:21!null v:22 w:23!null x:24!null y:25 - │ ├── with-scan &1 - │ │ ├── columns: k:21!null v:22 w:23!null x:24!null y:25 - │ │ └── mapping: - │ │ ├── uniq.k:7 => k:21 - │ │ ├── uniq.v:8 => v:22 - │ │ ├── w_new:13 => w:23 - │ │ ├── x_new:14 => x:24 - │ │ └── uniq.y:11 => y:25 - │ ├── scan uniq - │ │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 - │ └── filters - │ ├── w:23 = uniq.w:17 - │ └── k:21 != uniq.k:15 + │ └── project + │ ├── columns: w:23!null + │ └── semi-join (hash) + │ ├── columns: k:21!null v:22 w:23!null x:24!null y:25 + │ ├── with-scan &1 + │ │ ├── columns: k:21!null v:22 w:23!null x:24!null y:25 + │ │ └── mapping: + │ │ ├── uniq.k:7 => k:21 + │ │ ├── uniq.v:8 => v:22 + │ │ ├── w_new:13 => w:23 + │ │ ├── x_new:14 => x:24 + │ │ └── uniq.y:11 => y:25 + │ ├── scan uniq + │ │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 + │ └── filters + │ ├── w:23 = uniq.w:17 + │ └── k:21 != uniq.k:15 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:32!null v:33 w:34!null x:35!null y:36 - ├── with-scan &1 - │ ├── columns: k:32!null v:33 w:34!null x:35!null y:36 - │ └── mapping: - │ ├── uniq.k:7 => k:32 - │ ├── uniq.v:8 => v:33 - │ ├── w_new:13 => w:34 - │ ├── x_new:14 => x:35 - │ └── uniq.y:11 => y:36 - ├── scan uniq - │ └── columns: uniq.k:26!null uniq.v:27 uniq.w:28 uniq.x:29 uniq.y:30 - └── filters - ├── x:35 = uniq.x:29 - ├── y:36 = uniq.y:30 - └── k:32 != uniq.k:26 + └── project + ├── columns: x:35!null y:36 + └── semi-join (hash) + ├── columns: k:32!null v:33 w:34!null x:35!null y:36 + ├── with-scan &1 + │ ├── columns: k:32!null v:33 w:34!null x:35!null y:36 + │ └── mapping: + │ ├── uniq.k:7 => k:32 + │ ├── uniq.v:8 => v:33 + │ ├── w_new:13 => w:34 + │ ├── x_new:14 => x:35 + │ └── uniq.y:11 => y:36 + ├── scan uniq + │ └── columns: uniq.k:26!null uniq.v:27 uniq.w:28 uniq.x:29 uniq.y:30 + └── filters + ├── x:35 = uniq.x:29 + ├── y:36 = uniq.y:30 + └── k:32 != uniq.k:26 # No need to plan checks for w since it's always null. build @@ -82,22 +86,24 @@ update uniq │ └── 1 [as=x_new:14] └── unique-checks └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:21!null v:22 w:23 x:24!null y:25 - ├── with-scan &1 - │ ├── columns: k:21!null v:22 w:23 x:24!null y:25 - │ └── mapping: - │ ├── uniq.k:7 => k:21 - │ ├── uniq.v:8 => v:22 - │ ├── w_new:13 => w:23 - │ ├── x_new:14 => x:24 - │ └── uniq.y:11 => y:25 - ├── scan uniq - │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 - └── filters - ├── x:24 = uniq.x:18 - ├── y:25 = uniq.y:19 - └── k:21 != uniq.k:15 + └── project + ├── columns: x:24!null y:25 + └── semi-join (hash) + ├── columns: k:21!null v:22 w:23 x:24!null y:25 + ├── with-scan &1 + │ ├── columns: k:21!null v:22 w:23 x:24!null y:25 + │ └── mapping: + │ ├── uniq.k:7 => k:21 + │ ├── uniq.v:8 => v:22 + │ ├── w_new:13 => w:23 + │ ├── x_new:14 => x:24 + │ └── uniq.y:11 => y:25 + ├── scan uniq + │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 + └── filters + ├── x:24 = uniq.x:18 + ├── y:25 = uniq.y:19 + └── k:21 != uniq.k:15 # No need to plan checks for x,y since x is always null. # Also update the primary key. @@ -122,21 +128,23 @@ update uniq │ └── NULL::INT8 [as=x_new:15] └── unique-checks └── unique-checks-item: uniq(w) - └── semi-join (hash) - ├── columns: k:22!null v:23 w:24!null x:25 y:26 - ├── with-scan &1 - │ ├── columns: k:22!null v:23 w:24!null x:25 y:26 - │ └── mapping: - │ ├── k_new:13 => k:22 - │ ├── uniq.v:8 => v:23 - │ ├── w_new:14 => w:24 - │ ├── x_new:15 => x:25 - │ └── uniq.y:11 => y:26 - ├── scan uniq - │ └── columns: uniq.k:16!null uniq.v:17 uniq.w:18 uniq.x:19 uniq.y:20 - └── filters - ├── w:24 = uniq.w:18 - └── k:22 != uniq.k:16 + └── project + ├── columns: w:24!null + └── semi-join (hash) + ├── columns: k:22!null v:23 w:24!null x:25 y:26 + ├── with-scan &1 + │ ├── columns: k:22!null v:23 w:24!null x:25 y:26 + │ └── mapping: + │ ├── k_new:13 => k:22 + │ ├── uniq.v:8 => v:23 + │ ├── w_new:14 => w:24 + │ ├── x_new:15 => x:25 + │ └── uniq.y:11 => y:26 + ├── scan uniq + │ └── columns: uniq.k:16!null uniq.v:17 uniq.w:18 uniq.x:19 uniq.y:20 + └── filters + ├── w:24 = uniq.w:18 + └── k:22 != uniq.k:16 # No need to plan checks for x,y since y is always null. build @@ -162,21 +170,23 @@ update uniq │ └── NULL::INT8 [as=y_new:14] └── unique-checks └── unique-checks-item: uniq(w) - └── semi-join (hash) - ├── columns: k:21!null v:22 w:23!null x:24 y:25 - ├── with-scan &1 - │ ├── columns: k:21!null v:22 w:23!null x:24 y:25 - │ └── mapping: - │ ├── uniq.k:7 => k:21 - │ ├── uniq.v:8 => v:22 - │ ├── w_new:13 => w:23 - │ ├── uniq.x:10 => x:24 - │ └── y_new:14 => y:25 - ├── scan uniq - │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 - └── filters - ├── w:23 = uniq.w:17 - └── k:21 != uniq.k:15 + └── project + ├── columns: w:23!null + └── semi-join (hash) + ├── columns: k:21!null v:22 w:23!null x:24 y:25 + ├── with-scan &1 + │ ├── columns: k:21!null v:22 w:23!null x:24 y:25 + │ └── mapping: + │ ├── uniq.k:7 => k:21 + │ ├── uniq.v:8 => v:22 + │ ├── w_new:13 => w:23 + │ ├── uniq.x:10 => x:24 + │ └── y_new:14 => y:25 + ├── scan uniq + │ └── columns: uniq.k:15!null uniq.v:16 uniq.w:17 uniq.x:18 uniq.y:19 + └── filters + ├── w:23 = uniq.w:17 + └── k:21 != uniq.k:15 # No need to plan checks since none of the columns requiring checks are updated. build @@ -248,38 +258,42 @@ update uniq │ └── other.crdb_internal_mvcc_timestamp:19 └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:26!null v:27 w:28!null x:29 y:30 - │ ├── with-scan &1 - │ │ ├── columns: k:26!null v:27 w:28!null x:29 y:30 - │ │ └── mapping: - │ │ ├── uniq.k:7 => k:26 - │ │ ├── uniq.v:8 => v:27 - │ │ ├── other.w:15 => w:28 - │ │ ├── other.x:16 => x:29 - │ │ └── uniq.y:11 => y:30 - │ ├── scan uniq - │ │ └── columns: uniq.k:20!null uniq.v:21 uniq.w:22 uniq.x:23 uniq.y:24 - │ └── filters - │ ├── w:28 = uniq.w:22 - │ └── k:26 != uniq.k:20 + │ └── project + │ ├── columns: w:28!null + │ └── semi-join (hash) + │ ├── columns: k:26!null v:27 w:28!null x:29 y:30 + │ ├── with-scan &1 + │ │ ├── columns: k:26!null v:27 w:28!null x:29 y:30 + │ │ └── mapping: + │ │ ├── uniq.k:7 => k:26 + │ │ ├── uniq.v:8 => v:27 + │ │ ├── other.w:15 => w:28 + │ │ ├── other.x:16 => x:29 + │ │ └── uniq.y:11 => y:30 + │ ├── scan uniq + │ │ └── columns: uniq.k:20!null uniq.v:21 uniq.w:22 uniq.x:23 uniq.y:24 + │ └── filters + │ ├── w:28 = uniq.w:22 + │ └── k:26 != uniq.k:20 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:37!null v:38 w:39!null x:40 y:41 - ├── with-scan &1 - │ ├── columns: k:37!null v:38 w:39!null x:40 y:41 - │ └── mapping: - │ ├── uniq.k:7 => k:37 - │ ├── uniq.v:8 => v:38 - │ ├── other.w:15 => w:39 - │ ├── other.x:16 => x:40 - │ └── uniq.y:11 => y:41 - ├── scan uniq - │ └── columns: uniq.k:31!null uniq.v:32 uniq.w:33 uniq.x:34 uniq.y:35 - └── filters - ├── x:40 = uniq.x:34 - ├── y:41 = uniq.y:35 - └── k:37 != uniq.k:31 + └── project + ├── columns: x:40 y:41 + └── semi-join (hash) + ├── columns: k:37!null v:38 w:39!null x:40 y:41 + ├── with-scan &1 + │ ├── columns: k:37!null v:38 w:39!null x:40 y:41 + │ └── mapping: + │ ├── uniq.k:7 => k:37 + │ ├── uniq.v:8 => v:38 + │ ├── other.w:15 => w:39 + │ ├── other.x:16 => x:40 + │ └── uniq.y:11 => y:41 + ├── scan uniq + │ └── columns: uniq.k:31!null uniq.v:32 uniq.w:33 uniq.x:34 uniq.y:35 + └── filters + ├── x:40 = uniq.x:34 + ├── y:41 = uniq.y:35 + └── k:37 != uniq.k:31 exec-ddl CREATE TABLE uniq_overlaps_pk ( @@ -325,52 +339,58 @@ update uniq_overlaps_pk │ └── 4 [as=d_new:14] └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:20!null b:21!null c:22!null d:23!null - │ ├── with-scan &1 - │ │ ├── columns: a:20!null b:21!null c:22!null d:23!null - │ │ └── mapping: - │ │ ├── a_new:11 => a:20 - │ │ ├── b_new:12 => b:21 - │ │ ├── c_new:13 => c:22 - │ │ └── d_new:14 => d:23 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:15!null uniq_overlaps_pk.b:16!null uniq_overlaps_pk.c:17 uniq_overlaps_pk.d:18 - │ └── filters - │ ├── b:21 = uniq_overlaps_pk.b:16 - │ ├── c:22 = uniq_overlaps_pk.c:17 - │ └── a:20 != uniq_overlaps_pk.a:15 + │ └── project + │ ├── columns: b:21!null c:22!null + │ └── semi-join (hash) + │ ├── columns: a:20!null b:21!null c:22!null d:23!null + │ ├── with-scan &1 + │ │ ├── columns: a:20!null b:21!null c:22!null d:23!null + │ │ └── mapping: + │ │ ├── a_new:11 => a:20 + │ │ ├── b_new:12 => b:21 + │ │ ├── c_new:13 => c:22 + │ │ └── d_new:14 => d:23 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:15!null uniq_overlaps_pk.b:16!null uniq_overlaps_pk.c:17 uniq_overlaps_pk.d:18 + │ └── filters + │ ├── b:21 = uniq_overlaps_pk.b:16 + │ ├── c:22 = uniq_overlaps_pk.c:17 + │ └── a:20 != uniq_overlaps_pk.a:15 ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:29!null b:30!null c:31!null d:32!null - │ ├── with-scan &1 - │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null - │ │ └── mapping: - │ │ ├── a_new:11 => a:29 - │ │ ├── b_new:12 => b:30 - │ │ ├── c_new:13 => c:31 - │ │ └── d_new:14 => d:32 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:24!null uniq_overlaps_pk.b:25!null uniq_overlaps_pk.c:26 uniq_overlaps_pk.d:27 - │ └── filters - │ ├── a:29 = uniq_overlaps_pk.a:24 - │ └── b:30 != uniq_overlaps_pk.b:25 + │ └── project + │ ├── columns: a:29!null + │ └── semi-join (hash) + │ ├── columns: a:29!null b:30!null c:31!null d:32!null + │ ├── with-scan &1 + │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null + │ │ └── mapping: + │ │ ├── a_new:11 => a:29 + │ │ ├── b_new:12 => b:30 + │ │ ├── c_new:13 => c:31 + │ │ └── d_new:14 => d:32 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:24!null uniq_overlaps_pk.b:25!null uniq_overlaps_pk.c:26 uniq_overlaps_pk.d:27 + │ └── filters + │ ├── a:29 = uniq_overlaps_pk.a:24 + │ └── b:30 != uniq_overlaps_pk.b:25 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:38!null b:39!null c:40!null d:41!null - ├── with-scan &1 - │ ├── columns: a:38!null b:39!null c:40!null d:41!null - │ └── mapping: - │ ├── a_new:11 => a:38 - │ ├── b_new:12 => b:39 - │ ├── c_new:13 => c:40 - │ └── d_new:14 => d:41 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:33!null uniq_overlaps_pk.b:34!null uniq_overlaps_pk.c:35 uniq_overlaps_pk.d:36 - └── filters - ├── c:40 = uniq_overlaps_pk.c:35 - ├── d:41 = uniq_overlaps_pk.d:36 - └── (a:38 != uniq_overlaps_pk.a:33) OR (b:39 != uniq_overlaps_pk.b:34) + └── project + ├── columns: c:40!null d:41!null + └── semi-join (hash) + ├── columns: a:38!null b:39!null c:40!null d:41!null + ├── with-scan &1 + │ ├── columns: a:38!null b:39!null c:40!null d:41!null + │ └── mapping: + │ ├── a_new:11 => a:38 + │ ├── b_new:12 => b:39 + │ ├── c_new:13 => c:40 + │ └── d_new:14 => d:41 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:33!null uniq_overlaps_pk.b:34!null uniq_overlaps_pk.c:35 uniq_overlaps_pk.d:36 + └── filters + ├── c:40 = uniq_overlaps_pk.c:35 + ├── d:41 = uniq_overlaps_pk.d:36 + └── (a:38 != uniq_overlaps_pk.a:33) OR (b:39 != uniq_overlaps_pk.b:34) # Update with non-constant input. # No need to add a check for b,c since those columns weren't updated. @@ -419,36 +439,40 @@ update uniq_overlaps_pk │ └── other.crdb_internal_mvcc_timestamp:17 └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:23 b:24!null c:25 d:26 - │ ├── with-scan &1 - │ │ ├── columns: a:23 b:24!null c:25 d:26 - │ │ └── mapping: - │ │ ├── k:11 => a:23 - │ │ ├── uniq_overlaps_pk.b:7 => b:24 - │ │ ├── uniq_overlaps_pk.c:8 => c:25 - │ │ └── v:12 => d:26 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:18!null uniq_overlaps_pk.b:19!null uniq_overlaps_pk.c:20 uniq_overlaps_pk.d:21 - │ └── filters - │ ├── a:23 = uniq_overlaps_pk.a:18 - │ └── b:24 != uniq_overlaps_pk.b:19 + │ └── project + │ ├── columns: a:23 + │ └── semi-join (hash) + │ ├── columns: a:23 b:24!null c:25 d:26 + │ ├── with-scan &1 + │ │ ├── columns: a:23 b:24!null c:25 d:26 + │ │ └── mapping: + │ │ ├── k:11 => a:23 + │ │ ├── uniq_overlaps_pk.b:7 => b:24 + │ │ ├── uniq_overlaps_pk.c:8 => c:25 + │ │ └── v:12 => d:26 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:18!null uniq_overlaps_pk.b:19!null uniq_overlaps_pk.c:20 uniq_overlaps_pk.d:21 + │ └── filters + │ ├── a:23 = uniq_overlaps_pk.a:18 + │ └── b:24 != uniq_overlaps_pk.b:19 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:32 b:33!null c:34 d:35 - ├── with-scan &1 - │ ├── columns: a:32 b:33!null c:34 d:35 - │ └── mapping: - │ ├── k:11 => a:32 - │ ├── uniq_overlaps_pk.b:7 => b:33 - │ ├── uniq_overlaps_pk.c:8 => c:34 - │ └── v:12 => d:35 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:27!null uniq_overlaps_pk.b:28!null uniq_overlaps_pk.c:29 uniq_overlaps_pk.d:30 - └── filters - ├── c:34 = uniq_overlaps_pk.c:29 - ├── d:35 = uniq_overlaps_pk.d:30 - └── (a:32 != uniq_overlaps_pk.a:27) OR (b:33 != uniq_overlaps_pk.b:28) + └── project + ├── columns: c:34 d:35 + └── semi-join (hash) + ├── columns: a:32 b:33!null c:34 d:35 + ├── with-scan &1 + │ ├── columns: a:32 b:33!null c:34 d:35 + │ └── mapping: + │ ├── k:11 => a:32 + │ ├── uniq_overlaps_pk.b:7 => b:33 + │ ├── uniq_overlaps_pk.c:8 => c:34 + │ └── v:12 => d:35 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:27!null uniq_overlaps_pk.b:28!null uniq_overlaps_pk.c:29 uniq_overlaps_pk.d:30 + └── filters + ├── c:34 = uniq_overlaps_pk.c:29 + ├── d:35 = uniq_overlaps_pk.d:30 + └── (a:32 != uniq_overlaps_pk.a:27) OR (b:33 != uniq_overlaps_pk.b:28) exec-ddl CREATE TABLE uniq_hidden_pk ( @@ -482,39 +506,43 @@ update uniq_hidden_pk │ └── 1 [as=a_new:13] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:20!null b:21 c:22 d:23 rowid:24!null - │ ├── with-scan &1 - │ │ ├── columns: a:20!null b:21 c:22 d:23 rowid:24!null - │ │ └── mapping: - │ │ ├── a_new:13 => a:20 - │ │ ├── uniq_hidden_pk.b:8 => b:21 - │ │ ├── uniq_hidden_pk.c:9 => c:22 - │ │ ├── uniq_hidden_pk.d:10 => d:23 - │ │ └── uniq_hidden_pk.rowid:11 => rowid:24 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:14 uniq_hidden_pk.b:15 uniq_hidden_pk.c:16 uniq_hidden_pk.d:17 uniq_hidden_pk.rowid:18!null - │ └── filters - │ ├── a:20 = uniq_hidden_pk.a:14 - │ ├── b:21 = uniq_hidden_pk.b:15 - │ ├── d:23 = uniq_hidden_pk.d:17 - │ └── rowid:24 != uniq_hidden_pk.rowid:18 + │ └── project + │ ├── columns: a:20!null b:21 d:23 + │ └── semi-join (hash) + │ ├── columns: a:20!null b:21 c:22 d:23 rowid:24!null + │ ├── with-scan &1 + │ │ ├── columns: a:20!null b:21 c:22 d:23 rowid:24!null + │ │ └── mapping: + │ │ ├── a_new:13 => a:20 + │ │ ├── uniq_hidden_pk.b:8 => b:21 + │ │ ├── uniq_hidden_pk.c:9 => c:22 + │ │ ├── uniq_hidden_pk.d:10 => d:23 + │ │ └── uniq_hidden_pk.rowid:11 => rowid:24 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:14 uniq_hidden_pk.b:15 uniq_hidden_pk.c:16 uniq_hidden_pk.d:17 uniq_hidden_pk.rowid:18!null + │ └── filters + │ ├── a:20 = uniq_hidden_pk.a:14 + │ ├── b:21 = uniq_hidden_pk.b:15 + │ ├── d:23 = uniq_hidden_pk.d:17 + │ └── rowid:24 != uniq_hidden_pk.rowid:18 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:31!null b:32 c:33 d:34 rowid:35!null - ├── with-scan &1 - │ ├── columns: a:31!null b:32 c:33 d:34 rowid:35!null - │ └── mapping: - │ ├── a_new:13 => a:31 - │ ├── uniq_hidden_pk.b:8 => b:32 - │ ├── uniq_hidden_pk.c:9 => c:33 - │ ├── uniq_hidden_pk.d:10 => d:34 - │ └── uniq_hidden_pk.rowid:11 => rowid:35 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:25 uniq_hidden_pk.b:26 uniq_hidden_pk.c:27 uniq_hidden_pk.d:28 uniq_hidden_pk.rowid:29!null - └── filters - ├── a:31 = uniq_hidden_pk.a:25 - └── rowid:35 != uniq_hidden_pk.rowid:29 + └── project + ├── columns: a:31!null + └── semi-join (hash) + ├── columns: a:31!null b:32 c:33 d:34 rowid:35!null + ├── with-scan &1 + │ ├── columns: a:31!null b:32 c:33 d:34 rowid:35!null + │ └── mapping: + │ ├── a_new:13 => a:31 + │ ├── uniq_hidden_pk.b:8 => b:32 + │ ├── uniq_hidden_pk.c:9 => c:33 + │ ├── uniq_hidden_pk.d:10 => d:34 + │ └── uniq_hidden_pk.rowid:11 => rowid:35 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:25 uniq_hidden_pk.b:26 uniq_hidden_pk.c:27 uniq_hidden_pk.d:28 uniq_hidden_pk.rowid:29!null + └── filters + ├── a:31 = uniq_hidden_pk.a:25 + └── rowid:35 != uniq_hidden_pk.rowid:29 # Update with non-constant input. # No need to add a check for b,c since those columns weren't updated. @@ -537,39 +565,43 @@ update uniq_hidden_pk │ └── filters (true) └── unique-checks ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:26 b:27 c:28 d:29 rowid:30!null - │ ├── with-scan &1 - │ │ ├── columns: a:26 b:27 c:28 d:29 rowid:30!null - │ │ └── mapping: - │ │ ├── k:13 => a:26 - │ │ ├── uniq_hidden_pk.b:8 => b:27 - │ │ ├── uniq_hidden_pk.c:9 => c:28 - │ │ ├── uniq_hidden_pk.d:10 => d:29 - │ │ └── uniq_hidden_pk.rowid:11 => rowid:30 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:20 uniq_hidden_pk.b:21 uniq_hidden_pk.c:22 uniq_hidden_pk.d:23 uniq_hidden_pk.rowid:24!null - │ └── filters - │ ├── a:26 = uniq_hidden_pk.a:20 - │ ├── b:27 = uniq_hidden_pk.b:21 - │ ├── d:29 = uniq_hidden_pk.d:23 - │ └── rowid:30 != uniq_hidden_pk.rowid:24 + │ └── project + │ ├── columns: a:26 b:27 d:29 + │ └── semi-join (hash) + │ ├── columns: a:26 b:27 c:28 d:29 rowid:30!null + │ ├── with-scan &1 + │ │ ├── columns: a:26 b:27 c:28 d:29 rowid:30!null + │ │ └── mapping: + │ │ ├── k:13 => a:26 + │ │ ├── uniq_hidden_pk.b:8 => b:27 + │ │ ├── uniq_hidden_pk.c:9 => c:28 + │ │ ├── uniq_hidden_pk.d:10 => d:29 + │ │ └── uniq_hidden_pk.rowid:11 => rowid:30 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:20 uniq_hidden_pk.b:21 uniq_hidden_pk.c:22 uniq_hidden_pk.d:23 uniq_hidden_pk.rowid:24!null + │ └── filters + │ ├── a:26 = uniq_hidden_pk.a:20 + │ ├── b:27 = uniq_hidden_pk.b:21 + │ ├── d:29 = uniq_hidden_pk.d:23 + │ └── rowid:30 != uniq_hidden_pk.rowid:24 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:37 b:38 c:39 d:40 rowid:41!null - ├── with-scan &1 - │ ├── columns: a:37 b:38 c:39 d:40 rowid:41!null - │ └── mapping: - │ ├── k:13 => a:37 - │ ├── uniq_hidden_pk.b:8 => b:38 - │ ├── uniq_hidden_pk.c:9 => c:39 - │ ├── uniq_hidden_pk.d:10 => d:40 - │ └── uniq_hidden_pk.rowid:11 => rowid:41 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:31 uniq_hidden_pk.b:32 uniq_hidden_pk.c:33 uniq_hidden_pk.d:34 uniq_hidden_pk.rowid:35!null - └── filters - ├── a:37 = uniq_hidden_pk.a:31 - └── rowid:41 != uniq_hidden_pk.rowid:35 + └── project + ├── columns: a:37 + └── semi-join (hash) + ├── columns: a:37 b:38 c:39 d:40 rowid:41!null + ├── with-scan &1 + │ ├── columns: a:37 b:38 c:39 d:40 rowid:41!null + │ └── mapping: + │ ├── k:13 => a:37 + │ ├── uniq_hidden_pk.b:8 => b:38 + │ ├── uniq_hidden_pk.c:9 => c:39 + │ ├── uniq_hidden_pk.d:10 => d:40 + │ └── uniq_hidden_pk.rowid:11 => rowid:41 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:31 uniq_hidden_pk.b:32 uniq_hidden_pk.c:33 uniq_hidden_pk.d:34 uniq_hidden_pk.rowid:35!null + └── filters + ├── a:37 = uniq_hidden_pk.a:31 + └── rowid:41 != uniq_hidden_pk.rowid:35 exec-ddl CREATE TABLE uniq_partial ( @@ -599,22 +631,24 @@ update uniq_partial │ └── 1 [as=a_new:11] └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:17!null a:18!null b:19 c:20 - ├── with-scan &1 - │ ├── columns: k:17!null a:18!null b:19 c:20 - │ └── mapping: - │ ├── uniq_partial.k:6 => k:17 - │ ├── a_new:11 => a:18 - │ ├── uniq_partial.b:8 => b:19 - │ └── uniq_partial.c:9 => c:20 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 uniq_partial.c:15 - └── filters - ├── a:18 = uniq_partial.a:13 - ├── b:19 > 0 - ├── uniq_partial.b:14 > 0 - └── k:17 != uniq_partial.k:12 + └── project + ├── columns: a:18!null + └── semi-join (hash) + ├── columns: k:17!null a:18!null b:19 c:20 + ├── with-scan &1 + │ ├── columns: k:17!null a:18!null b:19 c:20 + │ └── mapping: + │ ├── uniq_partial.k:6 => k:17 + │ ├── a_new:11 => a:18 + │ ├── uniq_partial.b:8 => b:19 + │ └── uniq_partial.c:9 => c:20 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 uniq_partial.c:15 + └── filters + ├── a:18 = uniq_partial.a:13 + ├── b:19 > 0 + ├── uniq_partial.b:14 > 0 + └── k:17 != uniq_partial.k:12 # Plan a check when a column in the predicate is updated. build @@ -634,22 +668,24 @@ update uniq_partial │ └── 1 [as=b_new:11] └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:17!null a:18 b:19!null c:20 - ├── with-scan &1 - │ ├── columns: k:17!null a:18 b:19!null c:20 - │ └── mapping: - │ ├── uniq_partial.k:6 => k:17 - │ ├── uniq_partial.a:7 => a:18 - │ ├── b_new:11 => b:19 - │ └── uniq_partial.c:9 => c:20 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 uniq_partial.c:15 - └── filters - ├── a:18 = uniq_partial.a:13 - ├── b:19 > 0 - ├── uniq_partial.b:14 > 0 - └── k:17 != uniq_partial.k:12 + └── project + ├── columns: a:18 + └── semi-join (hash) + ├── columns: k:17!null a:18 b:19!null c:20 + ├── with-scan &1 + │ ├── columns: k:17!null a:18 b:19!null c:20 + │ └── mapping: + │ ├── uniq_partial.k:6 => k:17 + │ ├── uniq_partial.a:7 => a:18 + │ ├── b_new:11 => b:19 + │ └── uniq_partial.c:9 => c:20 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 uniq_partial.c:15 + └── filters + ├── a:18 = uniq_partial.a:13 + ├── b:19 > 0 + ├── uniq_partial.b:14 > 0 + └── k:17 != uniq_partial.k:12 # No need to plan checks for a since it's always null. build @@ -752,22 +788,24 @@ update uniq_partial │ └── other.crdb_internal_mvcc_timestamp:17 └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:23!null a:24!null b:25 c:26 - ├── with-scan &1 - │ ├── columns: k:23!null a:24!null b:25 c:26 - │ └── mapping: - │ ├── uniq_partial.k:6 => k:23 - │ ├── w:13 => a:24 - │ ├── x:14 => b:25 - │ └── uniq_partial.c:9 => c:26 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:18!null uniq_partial.a:19 uniq_partial.b:20 uniq_partial.c:21 - └── filters - ├── a:24 = uniq_partial.a:19 - ├── b:25 > 0 - ├── uniq_partial.b:20 > 0 - └── k:23 != uniq_partial.k:18 + └── project + ├── columns: a:24!null + └── semi-join (hash) + ├── columns: k:23!null a:24!null b:25 c:26 + ├── with-scan &1 + │ ├── columns: k:23!null a:24!null b:25 c:26 + │ └── mapping: + │ ├── uniq_partial.k:6 => k:23 + │ ├── w:13 => a:24 + │ ├── x:14 => b:25 + │ └── uniq_partial.c:9 => c:26 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:18!null uniq_partial.a:19 uniq_partial.b:20 uniq_partial.c:21 + └── filters + ├── a:24 = uniq_partial.a:19 + ├── b:25 > 0 + ├── uniq_partial.b:20 > 0 + └── k:23 != uniq_partial.k:18 exec-ddl CREATE TABLE uniq_partial_overlaps_pk ( @@ -816,57 +854,63 @@ update uniq_partial_overlaps_pk │ └── 4 [as=d_new:14] └── unique-checks ├── unique-checks-item: uniq_partial_overlaps_pk(c) - │ └── semi-join (hash) - │ ├── columns: a:20!null b:21!null c:22!null d:23!null - │ ├── with-scan &1 - │ │ ├── columns: a:20!null b:21!null c:22!null d:23!null - │ │ └── mapping: - │ │ ├── a_new:11 => a:20 - │ │ ├── b_new:12 => b:21 - │ │ ├── c_new:13 => c:22 - │ │ └── d_new:14 => d:23 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:15!null uniq_partial_overlaps_pk.b:16!null uniq_partial_overlaps_pk.c:17 uniq_partial_overlaps_pk.d:18 - │ └── filters - │ ├── c:22 = uniq_partial_overlaps_pk.c:17 - │ ├── d:23 > 0 - │ ├── uniq_partial_overlaps_pk.d:18 > 0 - │ └── (a:20 != uniq_partial_overlaps_pk.a:15) OR (b:21 != uniq_partial_overlaps_pk.b:16) + │ └── project + │ ├── columns: c:22!null + │ └── semi-join (hash) + │ ├── columns: a:20!null b:21!null c:22!null d:23!null + │ ├── with-scan &1 + │ │ ├── columns: a:20!null b:21!null c:22!null d:23!null + │ │ └── mapping: + │ │ ├── a_new:11 => a:20 + │ │ ├── b_new:12 => b:21 + │ │ ├── c_new:13 => c:22 + │ │ └── d_new:14 => d:23 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:15!null uniq_partial_overlaps_pk.b:16!null uniq_partial_overlaps_pk.c:17 uniq_partial_overlaps_pk.d:18 + │ └── filters + │ ├── c:22 = uniq_partial_overlaps_pk.c:17 + │ ├── d:23 > 0 + │ ├── uniq_partial_overlaps_pk.d:18 > 0 + │ └── (a:20 != uniq_partial_overlaps_pk.a:15) OR (b:21 != uniq_partial_overlaps_pk.b:16) ├── unique-checks-item: uniq_partial_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:29!null b:30!null c:31!null d:32!null - │ ├── with-scan &1 - │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null - │ │ └── mapping: - │ │ ├── a_new:11 => a:29 - │ │ ├── b_new:12 => b:30 - │ │ ├── c_new:13 => c:31 - │ │ └── d_new:14 => d:32 - │ ├── scan uniq_partial_overlaps_pk - │ │ └── columns: uniq_partial_overlaps_pk.a:24!null uniq_partial_overlaps_pk.b:25!null uniq_partial_overlaps_pk.c:26 uniq_partial_overlaps_pk.d:27 - │ └── filters - │ ├── a:29 = uniq_partial_overlaps_pk.a:24 - │ ├── d:32 > 0 - │ ├── uniq_partial_overlaps_pk.d:27 > 0 - │ └── b:30 != uniq_partial_overlaps_pk.b:25 + │ └── project + │ ├── columns: a:29!null + │ └── semi-join (hash) + │ ├── columns: a:29!null b:30!null c:31!null d:32!null + │ ├── with-scan &1 + │ │ ├── columns: a:29!null b:30!null c:31!null d:32!null + │ │ └── mapping: + │ │ ├── a_new:11 => a:29 + │ │ ├── b_new:12 => b:30 + │ │ ├── c_new:13 => c:31 + │ │ └── d_new:14 => d:32 + │ ├── scan uniq_partial_overlaps_pk + │ │ └── columns: uniq_partial_overlaps_pk.a:24!null uniq_partial_overlaps_pk.b:25!null uniq_partial_overlaps_pk.c:26 uniq_partial_overlaps_pk.d:27 + │ └── filters + │ ├── a:29 = uniq_partial_overlaps_pk.a:24 + │ ├── d:32 > 0 + │ ├── uniq_partial_overlaps_pk.d:27 > 0 + │ └── b:30 != uniq_partial_overlaps_pk.b:25 └── unique-checks-item: uniq_partial_overlaps_pk(b,c) - └── semi-join (hash) - ├── columns: a:38!null b:39!null c:40!null d:41!null - ├── with-scan &1 - │ ├── columns: a:38!null b:39!null c:40!null d:41!null - │ └── mapping: - │ ├── a_new:11 => a:38 - │ ├── b_new:12 => b:39 - │ ├── c_new:13 => c:40 - │ └── d_new:14 => d:41 - ├── scan uniq_partial_overlaps_pk - │ └── columns: uniq_partial_overlaps_pk.a:33!null uniq_partial_overlaps_pk.b:34!null uniq_partial_overlaps_pk.c:35 uniq_partial_overlaps_pk.d:36 - └── filters - ├── b:39 = uniq_partial_overlaps_pk.b:34 - ├── c:40 = uniq_partial_overlaps_pk.c:35 - ├── d:41 > 0 - ├── uniq_partial_overlaps_pk.d:36 > 0 - └── a:38 != uniq_partial_overlaps_pk.a:33 + └── project + ├── columns: b:39!null c:40!null + └── semi-join (hash) + ├── columns: a:38!null b:39!null c:40!null d:41!null + ├── with-scan &1 + │ ├── columns: a:38!null b:39!null c:40!null d:41!null + │ └── mapping: + │ ├── a_new:11 => a:38 + │ ├── b_new:12 => b:39 + │ ├── c_new:13 => c:40 + │ └── d_new:14 => d:41 + ├── scan uniq_partial_overlaps_pk + │ └── columns: uniq_partial_overlaps_pk.a:33!null uniq_partial_overlaps_pk.b:34!null uniq_partial_overlaps_pk.c:35 uniq_partial_overlaps_pk.d:36 + └── filters + ├── b:39 = uniq_partial_overlaps_pk.b:34 + ├── c:40 = uniq_partial_overlaps_pk.c:35 + ├── d:41 > 0 + ├── uniq_partial_overlaps_pk.d:36 > 0 + └── a:38 != uniq_partial_overlaps_pk.a:33 # Update with non-constant input. # Do not build uniqueness checks when the primary key columns are a subset of @@ -916,22 +960,24 @@ update uniq_partial_overlaps_pk │ └── other.crdb_internal_mvcc_timestamp:17 └── unique-checks └── unique-checks-item: uniq_partial_overlaps_pk(a) - └── semi-join (hash) - ├── columns: a:23 b:24!null c:25 d:26 - ├── with-scan &1 - │ ├── columns: a:23 b:24!null c:25 d:26 - │ └── mapping: - │ ├── k:11 => a:23 - │ ├── uniq_partial_overlaps_pk.b:7 => b:24 - │ ├── uniq_partial_overlaps_pk.c:8 => c:25 - │ └── uniq_partial_overlaps_pk.d:9 => d:26 - ├── scan uniq_partial_overlaps_pk - │ └── columns: uniq_partial_overlaps_pk.a:18!null uniq_partial_overlaps_pk.b:19!null uniq_partial_overlaps_pk.c:20 uniq_partial_overlaps_pk.d:21 - └── filters - ├── a:23 = uniq_partial_overlaps_pk.a:18 - ├── d:26 > 0 - ├── uniq_partial_overlaps_pk.d:21 > 0 - └── b:24 != uniq_partial_overlaps_pk.b:19 + └── project + ├── columns: a:23 + └── semi-join (hash) + ├── columns: a:23 b:24!null c:25 d:26 + ├── with-scan &1 + │ ├── columns: a:23 b:24!null c:25 d:26 + │ └── mapping: + │ ├── k:11 => a:23 + │ ├── uniq_partial_overlaps_pk.b:7 => b:24 + │ ├── uniq_partial_overlaps_pk.c:8 => c:25 + │ └── uniq_partial_overlaps_pk.d:9 => d:26 + ├── scan uniq_partial_overlaps_pk + │ └── columns: uniq_partial_overlaps_pk.a:18!null uniq_partial_overlaps_pk.b:19!null uniq_partial_overlaps_pk.c:20 uniq_partial_overlaps_pk.d:21 + └── filters + ├── a:23 = uniq_partial_overlaps_pk.a:18 + ├── d:26 > 0 + ├── uniq_partial_overlaps_pk.d:21 > 0 + └── b:24 != uniq_partial_overlaps_pk.b:19 exec-ddl CREATE TABLE uniq_partial_hidden_pk ( @@ -960,21 +1006,23 @@ update uniq_partial_hidden_pk │ └── 1 [as=a_new:9] └── unique-checks └── unique-checks-item: uniq_partial_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:14!null b:15 rowid:16!null - ├── with-scan &1 - │ ├── columns: a:14!null b:15 rowid:16!null - │ └── mapping: - │ ├── a_new:9 => a:14 - │ ├── uniq_partial_hidden_pk.b:6 => b:15 - │ └── uniq_partial_hidden_pk.rowid:7 => rowid:16 - ├── scan uniq_partial_hidden_pk - │ └── columns: uniq_partial_hidden_pk.a:10 uniq_partial_hidden_pk.b:11 uniq_partial_hidden_pk.rowid:12!null - └── filters - ├── a:14 = uniq_partial_hidden_pk.a:10 - ├── b:15 > 0 - ├── uniq_partial_hidden_pk.b:11 > 0 - └── rowid:16 != uniq_partial_hidden_pk.rowid:12 + └── project + ├── columns: a:14!null + └── semi-join (hash) + ├── columns: a:14!null b:15 rowid:16!null + ├── with-scan &1 + │ ├── columns: a:14!null b:15 rowid:16!null + │ └── mapping: + │ ├── a_new:9 => a:14 + │ ├── uniq_partial_hidden_pk.b:6 => b:15 + │ └── uniq_partial_hidden_pk.rowid:7 => rowid:16 + ├── scan uniq_partial_hidden_pk + │ └── columns: uniq_partial_hidden_pk.a:10 uniq_partial_hidden_pk.b:11 uniq_partial_hidden_pk.rowid:12!null + └── filters + ├── a:14 = uniq_partial_hidden_pk.a:10 + ├── b:15 > 0 + ├── uniq_partial_hidden_pk.b:11 > 0 + └── rowid:16 != uniq_partial_hidden_pk.rowid:12 # Update with non-constant input. # Add inequality filters for the hidden primary key column. @@ -996,21 +1044,23 @@ update uniq_partial_hidden_pk │ └── filters (true) └── unique-checks └── unique-checks-item: uniq_partial_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:20 b:21 rowid:22!null - ├── with-scan &1 - │ ├── columns: a:20 b:21 rowid:22!null - │ └── mapping: - │ ├── k:9 => a:20 - │ ├── uniq_partial_hidden_pk.b:6 => b:21 - │ └── uniq_partial_hidden_pk.rowid:7 => rowid:22 - ├── scan uniq_partial_hidden_pk - │ └── columns: uniq_partial_hidden_pk.a:16 uniq_partial_hidden_pk.b:17 uniq_partial_hidden_pk.rowid:18!null - └── filters - ├── a:20 = uniq_partial_hidden_pk.a:16 - ├── b:21 > 0 - ├── uniq_partial_hidden_pk.b:17 > 0 - └── rowid:22 != uniq_partial_hidden_pk.rowid:18 + └── project + ├── columns: a:20 + └── semi-join (hash) + ├── columns: a:20 b:21 rowid:22!null + ├── with-scan &1 + │ ├── columns: a:20 b:21 rowid:22!null + │ └── mapping: + │ ├── k:9 => a:20 + │ ├── uniq_partial_hidden_pk.b:6 => b:21 + │ └── uniq_partial_hidden_pk.rowid:7 => rowid:22 + ├── scan uniq_partial_hidden_pk + │ └── columns: uniq_partial_hidden_pk.a:16 uniq_partial_hidden_pk.b:17 uniq_partial_hidden_pk.rowid:18!null + └── filters + ├── a:20 = uniq_partial_hidden_pk.a:16 + ├── b:21 > 0 + ├── uniq_partial_hidden_pk.b:17 > 0 + └── rowid:22 != uniq_partial_hidden_pk.rowid:18 exec-ddl CREATE TABLE uniq_computed_pk ( @@ -1076,33 +1126,35 @@ update uniq_computed_pk │ └── d_new:19::STRING [as=column21:21] └── unique-checks └── unique-checks-item: uniq_computed_pk(d) - └── semi-join (hash) - ├── columns: i:46!null s:47!null d:48!null c_i_expr:49!null c_s:50!null c_d:51!null c_d_expr:52!null - ├── with-scan &1 - │ ├── columns: i:46!null s:47!null d:48!null c_i_expr:49!null c_s:50!null c_d:51!null c_d_expr:52!null - │ └── mapping: - │ ├── i_new:17 => i:46 - │ ├── s_new:18 => s:47 - │ ├── d_new:19 => d:48 - │ ├── column20:20 => c_i_expr:49 - │ ├── s_new:18 => c_s:50 - │ ├── d_new:19 => c_d:51 - │ └── column21:21 => c_d_expr:52 - ├── project - │ ├── columns: uniq_computed_pk.c_s:42 uniq_computed_pk.i:38!null uniq_computed_pk.s:39 uniq_computed_pk.d:40 uniq_computed_pk.c_i_expr:41!null uniq_computed_pk.c_d:43 uniq_computed_pk.c_d_expr:44 - │ ├── scan uniq_computed_pk - │ │ ├── columns: uniq_computed_pk.i:38!null uniq_computed_pk.s:39 uniq_computed_pk.d:40 uniq_computed_pk.c_i_expr:41!null uniq_computed_pk.c_d:43 uniq_computed_pk.c_d_expr:44 - │ │ └── computed column expressions - │ │ ├── uniq_computed_pk.c_i_expr:41 - │ │ │ └── CASE WHEN uniq_computed_pk.i:38 < 0 THEN 'foo' ELSE 'bar' END - │ │ ├── uniq_computed_pk.c_s:42 - │ │ │ └── uniq_computed_pk.s:39 - │ │ ├── uniq_computed_pk.c_d:43 - │ │ │ └── uniq_computed_pk.d:40 - │ │ └── uniq_computed_pk.c_d_expr:44 - │ │ └── uniq_computed_pk.d:40::STRING - │ └── projections - │ └── uniq_computed_pk.s:39 [as=uniq_computed_pk.c_s:42] - └── filters - ├── d:48 = uniq_computed_pk.d:40 - └── (i:46 != uniq_computed_pk.i:38) OR (c_i_expr:49 != uniq_computed_pk.c_i_expr:41) + └── project + ├── columns: d:48!null + └── semi-join (hash) + ├── columns: i:46!null s:47!null d:48!null c_i_expr:49!null c_s:50!null c_d:51!null c_d_expr:52!null + ├── with-scan &1 + │ ├── columns: i:46!null s:47!null d:48!null c_i_expr:49!null c_s:50!null c_d:51!null c_d_expr:52!null + │ └── mapping: + │ ├── i_new:17 => i:46 + │ ├── s_new:18 => s:47 + │ ├── d_new:19 => d:48 + │ ├── column20:20 => c_i_expr:49 + │ ├── s_new:18 => c_s:50 + │ ├── d_new:19 => c_d:51 + │ └── column21:21 => c_d_expr:52 + ├── project + │ ├── columns: uniq_computed_pk.c_s:42 uniq_computed_pk.i:38!null uniq_computed_pk.s:39 uniq_computed_pk.d:40 uniq_computed_pk.c_i_expr:41!null uniq_computed_pk.c_d:43 uniq_computed_pk.c_d_expr:44 + │ ├── scan uniq_computed_pk + │ │ ├── columns: uniq_computed_pk.i:38!null uniq_computed_pk.s:39 uniq_computed_pk.d:40 uniq_computed_pk.c_i_expr:41!null uniq_computed_pk.c_d:43 uniq_computed_pk.c_d_expr:44 + │ │ └── computed column expressions + │ │ ├── uniq_computed_pk.c_i_expr:41 + │ │ │ └── CASE WHEN uniq_computed_pk.i:38 < 0 THEN 'foo' ELSE 'bar' END + │ │ ├── uniq_computed_pk.c_s:42 + │ │ │ └── uniq_computed_pk.s:39 + │ │ ├── uniq_computed_pk.c_d:43 + │ │ │ └── uniq_computed_pk.d:40 + │ │ └── uniq_computed_pk.c_d_expr:44 + │ │ └── uniq_computed_pk.d:40::STRING + │ └── projections + │ └── uniq_computed_pk.s:39 [as=uniq_computed_pk.c_s:42] + └── filters + ├── d:48 = uniq_computed_pk.d:40 + └── (i:46 != uniq_computed_pk.i:38) OR (c_i_expr:49 != uniq_computed_pk.c_i_expr:41) diff --git a/pkg/sql/opt/optbuilder/testdata/unique-checks-upsert b/pkg/sql/opt/optbuilder/testdata/unique-checks-upsert index 0ce52179b695..c8bed47afb20 100644 --- a/pkg/sql/opt/optbuilder/testdata/unique-checks-upsert +++ b/pkg/sql/opt/optbuilder/testdata/unique-checks-upsert @@ -62,38 +62,42 @@ upsert uniq │ └── CASE WHEN uniq.k:12 IS NULL THEN column1:7 ELSE uniq.k:12 END [as=upsert_k:18] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:25 v:26!null w:27!null x:28!null y:29!null - │ ├── with-scan &1 - │ │ ├── columns: k:25 v:26!null w:27!null x:28!null y:29!null - │ │ └── mapping: - │ │ ├── upsert_k:18 => k:25 - │ │ ├── column2:8 => v:26 - │ │ ├── column3:9 => w:27 - │ │ ├── column4:10 => x:28 - │ │ └── column5:11 => y:29 - │ ├── scan uniq - │ │ └── columns: uniq.k:19!null uniq.v:20 uniq.w:21 uniq.x:22 uniq.y:23 - │ └── filters - │ ├── w:27 = uniq.w:21 - │ └── k:25 != uniq.k:19 + │ └── project + │ ├── columns: w:27!null + │ └── semi-join (hash) + │ ├── columns: k:25 v:26!null w:27!null x:28!null y:29!null + │ ├── with-scan &1 + │ │ ├── columns: k:25 v:26!null w:27!null x:28!null y:29!null + │ │ └── mapping: + │ │ ├── upsert_k:18 => k:25 + │ │ ├── column2:8 => v:26 + │ │ ├── column3:9 => w:27 + │ │ ├── column4:10 => x:28 + │ │ └── column5:11 => y:29 + │ ├── scan uniq + │ │ └── columns: uniq.k:19!null uniq.v:20 uniq.w:21 uniq.x:22 uniq.y:23 + │ └── filters + │ ├── w:27 = uniq.w:21 + │ └── k:25 != uniq.k:19 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:36 v:37!null w:38!null x:39!null y:40!null - ├── with-scan &1 - │ ├── columns: k:36 v:37!null w:38!null x:39!null y:40!null - │ └── mapping: - │ ├── upsert_k:18 => k:36 - │ ├── column2:8 => v:37 - │ ├── column3:9 => w:38 - │ ├── column4:10 => x:39 - │ └── column5:11 => y:40 - ├── scan uniq - │ └── columns: uniq.k:30!null uniq.v:31 uniq.w:32 uniq.x:33 uniq.y:34 - └── filters - ├── x:39 = uniq.x:33 - ├── y:40 = uniq.y:34 - └── k:36 != uniq.k:30 + └── project + ├── columns: x:39!null y:40!null + └── semi-join (hash) + ├── columns: k:36 v:37!null w:38!null x:39!null y:40!null + ├── with-scan &1 + │ ├── columns: k:36 v:37!null w:38!null x:39!null y:40!null + │ └── mapping: + │ ├── upsert_k:18 => k:36 + │ ├── column2:8 => v:37 + │ ├── column3:9 => w:38 + │ ├── column4:10 => x:39 + │ └── column5:11 => y:40 + ├── scan uniq + │ └── columns: uniq.k:30!null uniq.v:31 uniq.w:32 uniq.x:33 uniq.y:34 + └── filters + ├── x:39 = uniq.x:33 + ├── y:40 = uniq.y:34 + └── k:36 != uniq.k:30 # TODO(rytaft): The default value for x is NULL, and we're not updating either # x or y. Therefore, we could avoid planning checks for (x,y) (see #58300). @@ -150,38 +154,42 @@ upsert uniq │ └── CASE WHEN uniq.k:12 IS NULL THEN column11:11 ELSE uniq.y:16 END [as=upsert_y:20] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:27 v:28!null w:29!null x:30 y:31 - │ ├── with-scan &1 - │ │ ├── columns: k:27 v:28!null w:29!null x:30 y:31 - │ │ └── mapping: - │ │ ├── upsert_k:18 => k:27 - │ │ ├── column2:8 => v:28 - │ │ ├── column3:9 => w:29 - │ │ ├── upsert_x:19 => x:30 - │ │ └── upsert_y:20 => y:31 - │ ├── scan uniq - │ │ └── columns: uniq.k:21!null uniq.v:22 uniq.w:23 uniq.x:24 uniq.y:25 - │ └── filters - │ ├── w:29 = uniq.w:23 - │ └── k:27 != uniq.k:21 + │ └── project + │ ├── columns: w:29!null + │ └── semi-join (hash) + │ ├── columns: k:27 v:28!null w:29!null x:30 y:31 + │ ├── with-scan &1 + │ │ ├── columns: k:27 v:28!null w:29!null x:30 y:31 + │ │ └── mapping: + │ │ ├── upsert_k:18 => k:27 + │ │ ├── column2:8 => v:28 + │ │ ├── column3:9 => w:29 + │ │ ├── upsert_x:19 => x:30 + │ │ └── upsert_y:20 => y:31 + │ ├── scan uniq + │ │ └── columns: uniq.k:21!null uniq.v:22 uniq.w:23 uniq.x:24 uniq.y:25 + │ └── filters + │ ├── w:29 = uniq.w:23 + │ └── k:27 != uniq.k:21 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:38 v:39!null w:40!null x:41 y:42 - ├── with-scan &1 - │ ├── columns: k:38 v:39!null w:40!null x:41 y:42 - │ └── mapping: - │ ├── upsert_k:18 => k:38 - │ ├── column2:8 => v:39 - │ ├── column3:9 => w:40 - │ ├── upsert_x:19 => x:41 - │ └── upsert_y:20 => y:42 - ├── scan uniq - │ └── columns: uniq.k:32!null uniq.v:33 uniq.w:34 uniq.x:35 uniq.y:36 - └── filters - ├── x:41 = uniq.x:35 - ├── y:42 = uniq.y:36 - └── k:38 != uniq.k:32 + └── project + ├── columns: x:41 y:42 + └── semi-join (hash) + ├── columns: k:38 v:39!null w:40!null x:41 y:42 + ├── with-scan &1 + │ ├── columns: k:38 v:39!null w:40!null x:41 y:42 + │ └── mapping: + │ ├── upsert_k:18 => k:38 + │ ├── column2:8 => v:39 + │ ├── column3:9 => w:40 + │ ├── upsert_x:19 => x:41 + │ └── upsert_y:20 => y:42 + ├── scan uniq + │ └── columns: uniq.k:32!null uniq.v:33 uniq.w:34 uniq.x:35 uniq.y:36 + └── filters + ├── x:41 = uniq.x:35 + ├── y:42 = uniq.y:36 + └── k:38 != uniq.k:32 # TODO(rytaft): No need to plan checks for w since it's always NULL. # We currently can't determine that w is always NULL since the function @@ -239,38 +247,42 @@ upsert uniq │ └── CASE WHEN uniq.k:12 IS NULL THEN column11:11 ELSE uniq.y:16 END [as=upsert_y:20] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:27 v:28 w:29 x:30 y:31 - │ ├── with-scan &1 - │ │ ├── columns: k:27 v:28 w:29 x:30 y:31 - │ │ └── mapping: - │ │ ├── upsert_k:18 => k:27 - │ │ ├── upsert_v:19 => v:28 - │ │ ├── column2:8 => w:29 - │ │ ├── column3:9 => x:30 - │ │ └── upsert_y:20 => y:31 - │ ├── scan uniq - │ │ └── columns: uniq.k:21!null uniq.v:22 uniq.w:23 uniq.x:24 uniq.y:25 - │ └── filters - │ ├── w:29 = uniq.w:23 - │ └── k:27 != uniq.k:21 + │ └── project + │ ├── columns: w:29 + │ └── semi-join (hash) + │ ├── columns: k:27 v:28 w:29 x:30 y:31 + │ ├── with-scan &1 + │ │ ├── columns: k:27 v:28 w:29 x:30 y:31 + │ │ └── mapping: + │ │ ├── upsert_k:18 => k:27 + │ │ ├── upsert_v:19 => v:28 + │ │ ├── column2:8 => w:29 + │ │ ├── column3:9 => x:30 + │ │ └── upsert_y:20 => y:31 + │ ├── scan uniq + │ │ └── columns: uniq.k:21!null uniq.v:22 uniq.w:23 uniq.x:24 uniq.y:25 + │ └── filters + │ ├── w:29 = uniq.w:23 + │ └── k:27 != uniq.k:21 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:38 v:39 w:40 x:41 y:42 - ├── with-scan &1 - │ ├── columns: k:38 v:39 w:40 x:41 y:42 - │ └── mapping: - │ ├── upsert_k:18 => k:38 - │ ├── upsert_v:19 => v:39 - │ ├── column2:8 => w:40 - │ ├── column3:9 => x:41 - │ └── upsert_y:20 => y:42 - ├── scan uniq - │ └── columns: uniq.k:32!null uniq.v:33 uniq.w:34 uniq.x:35 uniq.y:36 - └── filters - ├── x:41 = uniq.x:35 - ├── y:42 = uniq.y:36 - └── k:38 != uniq.k:32 + └── project + ├── columns: x:41 y:42 + └── semi-join (hash) + ├── columns: k:38 v:39 w:40 x:41 y:42 + ├── with-scan &1 + │ ├── columns: k:38 v:39 w:40 x:41 y:42 + │ └── mapping: + │ ├── upsert_k:18 => k:38 + │ ├── upsert_v:19 => v:39 + │ ├── column2:8 => w:40 + │ ├── column3:9 => x:41 + │ └── upsert_y:20 => y:42 + ├── scan uniq + │ └── columns: uniq.k:32!null uniq.v:33 uniq.w:34 uniq.x:35 uniq.y:36 + └── filters + ├── x:41 = uniq.x:35 + ├── y:42 = uniq.y:36 + └── k:38 != uniq.k:32 # Upsert with non-constant input. # TODO(rytaft): The default value for x is NULL, and we're not updating either @@ -328,38 +340,42 @@ upsert uniq │ └── CASE WHEN uniq.k:16 IS NULL THEN other.k:7 ELSE uniq.k:16 END [as=upsert_k:22] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:29 v:30 w:31!null x:32 y:33!null - │ ├── with-scan &1 - │ │ ├── columns: k:29 v:30 w:31!null x:32 y:33!null - │ │ └── mapping: - │ │ ├── upsert_k:22 => k:29 - │ │ ├── other.v:8 => v:30 - │ │ ├── other.w:9 => w:31 - │ │ ├── column14:14 => x:32 - │ │ └── column15:15 => y:33 - │ ├── scan uniq - │ │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 - │ └── filters - │ ├── w:31 = uniq.w:25 - │ └── k:29 != uniq.k:23 + │ └── project + │ ├── columns: w:31!null + │ └── semi-join (hash) + │ ├── columns: k:29 v:30 w:31!null x:32 y:33!null + │ ├── with-scan &1 + │ │ ├── columns: k:29 v:30 w:31!null x:32 y:33!null + │ │ └── mapping: + │ │ ├── upsert_k:22 => k:29 + │ │ ├── other.v:8 => v:30 + │ │ ├── other.w:9 => w:31 + │ │ ├── column14:14 => x:32 + │ │ └── column15:15 => y:33 + │ ├── scan uniq + │ │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 + │ └── filters + │ ├── w:31 = uniq.w:25 + │ └── k:29 != uniq.k:23 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:40 v:41 w:42!null x:43 y:44!null - ├── with-scan &1 - │ ├── columns: k:40 v:41 w:42!null x:43 y:44!null - │ └── mapping: - │ ├── upsert_k:22 => k:40 - │ ├── other.v:8 => v:41 - │ ├── other.w:9 => w:42 - │ ├── column14:14 => x:43 - │ └── column15:15 => y:44 - ├── scan uniq - │ └── columns: uniq.k:34!null uniq.v:35 uniq.w:36 uniq.x:37 uniq.y:38 - └── filters - ├── x:43 = uniq.x:37 - ├── y:44 = uniq.y:38 - └── k:40 != uniq.k:34 + └── project + ├── columns: x:43 y:44!null + └── semi-join (hash) + ├── columns: k:40 v:41 w:42!null x:43 y:44!null + ├── with-scan &1 + │ ├── columns: k:40 v:41 w:42!null x:43 y:44!null + │ └── mapping: + │ ├── upsert_k:22 => k:40 + │ ├── other.v:8 => v:41 + │ ├── other.w:9 => w:42 + │ ├── column14:14 => x:43 + │ └── column15:15 => y:44 + ├── scan uniq + │ └── columns: uniq.k:34!null uniq.v:35 uniq.w:36 uniq.x:37 uniq.y:38 + └── filters + ├── x:43 = uniq.x:37 + ├── y:44 = uniq.y:38 + └── k:40 != uniq.k:34 # On conflict do update with constant input. # TODO(rytaft): The default value for x is NULL, and we're not updating either @@ -421,38 +437,42 @@ upsert uniq │ └── CASE WHEN uniq.k:11 IS NULL THEN column10:10 ELSE uniq.y:15 END [as=upsert_y:22] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:29 v:30 w:31 x:32 y:33 - │ ├── with-scan &1 - │ │ ├── columns: k:29 v:30 w:31 x:32 y:33 - │ │ └── mapping: - │ │ ├── upsert_k:18 => k:29 - │ │ ├── upsert_v:19 => v:30 - │ │ ├── upsert_w:20 => w:31 - │ │ ├── upsert_x:21 => x:32 - │ │ └── upsert_y:22 => y:33 - │ ├── scan uniq - │ │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 - │ └── filters - │ ├── w:31 = uniq.w:25 - │ └── k:29 != uniq.k:23 + │ └── project + │ ├── columns: w:31 + │ └── semi-join (hash) + │ ├── columns: k:29 v:30 w:31 x:32 y:33 + │ ├── with-scan &1 + │ │ ├── columns: k:29 v:30 w:31 x:32 y:33 + │ │ └── mapping: + │ │ ├── upsert_k:18 => k:29 + │ │ ├── upsert_v:19 => v:30 + │ │ ├── upsert_w:20 => w:31 + │ │ ├── upsert_x:21 => x:32 + │ │ └── upsert_y:22 => y:33 + │ ├── scan uniq + │ │ └── columns: uniq.k:23!null uniq.v:24 uniq.w:25 uniq.x:26 uniq.y:27 + │ └── filters + │ ├── w:31 = uniq.w:25 + │ └── k:29 != uniq.k:23 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:40 v:41 w:42 x:43 y:44 - ├── with-scan &1 - │ ├── columns: k:40 v:41 w:42 x:43 y:44 - │ └── mapping: - │ ├── upsert_k:18 => k:40 - │ ├── upsert_v:19 => v:41 - │ ├── upsert_w:20 => w:42 - │ ├── upsert_x:21 => x:43 - │ └── upsert_y:22 => y:44 - ├── scan uniq - │ └── columns: uniq.k:34!null uniq.v:35 uniq.w:36 uniq.x:37 uniq.y:38 - └── filters - ├── x:43 = uniq.x:37 - ├── y:44 = uniq.y:38 - └── k:40 != uniq.k:34 + └── project + ├── columns: x:43 y:44 + └── semi-join (hash) + ├── columns: k:40 v:41 w:42 x:43 y:44 + ├── with-scan &1 + │ ├── columns: k:40 v:41 w:42 x:43 y:44 + │ └── mapping: + │ ├── upsert_k:18 => k:40 + │ ├── upsert_v:19 => v:41 + │ ├── upsert_w:20 => w:42 + │ ├── upsert_x:21 => x:43 + │ └── upsert_y:22 => y:44 + ├── scan uniq + │ └── columns: uniq.k:34!null uniq.v:35 uniq.w:36 uniq.x:37 uniq.y:38 + └── filters + ├── x:43 = uniq.x:37 + ├── y:44 = uniq.y:38 + └── k:40 != uniq.k:34 # On conflict do update with non-constant input. # TODO(rytaft): The default value for x is NULL, and we're not updating either @@ -513,38 +533,42 @@ upsert uniq │ └── CASE WHEN uniq.k:16 IS NULL THEN column15:15 ELSE uniq.y:20 END [as=upsert_y:27] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:34 v:35 w:36 x:37 y:38 - │ ├── with-scan &1 - │ │ ├── columns: k:34 v:35 w:36 x:37 y:38 - │ │ └── mapping: - │ │ ├── upsert_k:23 => k:34 - │ │ ├── upsert_v:24 => v:35 - │ │ ├── upsert_w:25 => w:36 - │ │ ├── upsert_x:26 => x:37 - │ │ └── upsert_y:27 => y:38 - │ ├── scan uniq - │ │ └── columns: uniq.k:28!null uniq.v:29 uniq.w:30 uniq.x:31 uniq.y:32 - │ └── filters - │ ├── w:36 = uniq.w:30 - │ └── k:34 != uniq.k:28 + │ └── project + │ ├── columns: w:36 + │ └── semi-join (hash) + │ ├── columns: k:34 v:35 w:36 x:37 y:38 + │ ├── with-scan &1 + │ │ ├── columns: k:34 v:35 w:36 x:37 y:38 + │ │ └── mapping: + │ │ ├── upsert_k:23 => k:34 + │ │ ├── upsert_v:24 => v:35 + │ │ ├── upsert_w:25 => w:36 + │ │ ├── upsert_x:26 => x:37 + │ │ └── upsert_y:27 => y:38 + │ ├── scan uniq + │ │ └── columns: uniq.k:28!null uniq.v:29 uniq.w:30 uniq.x:31 uniq.y:32 + │ └── filters + │ ├── w:36 = uniq.w:30 + │ └── k:34 != uniq.k:28 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:45 v:46 w:47 x:48 y:49 - ├── with-scan &1 - │ ├── columns: k:45 v:46 w:47 x:48 y:49 - │ └── mapping: - │ ├── upsert_k:23 => k:45 - │ ├── upsert_v:24 => v:46 - │ ├── upsert_w:25 => w:47 - │ ├── upsert_x:26 => x:48 - │ └── upsert_y:27 => y:49 - ├── scan uniq - │ └── columns: uniq.k:39!null uniq.v:40 uniq.w:41 uniq.x:42 uniq.y:43 - └── filters - ├── x:48 = uniq.x:42 - ├── y:49 = uniq.y:43 - └── k:45 != uniq.k:39 + └── project + ├── columns: x:48 y:49 + └── semi-join (hash) + ├── columns: k:45 v:46 w:47 x:48 y:49 + ├── with-scan &1 + │ ├── columns: k:45 v:46 w:47 x:48 y:49 + │ └── mapping: + │ ├── upsert_k:23 => k:45 + │ ├── upsert_v:24 => v:46 + │ ├── upsert_w:25 => w:47 + │ ├── upsert_x:26 => x:48 + │ └── upsert_y:27 => y:49 + ├── scan uniq + │ └── columns: uniq.k:39!null uniq.v:40 uniq.w:41 uniq.x:42 uniq.y:43 + └── filters + ├── x:48 = uniq.x:42 + ├── y:49 = uniq.y:43 + └── k:45 != uniq.k:39 # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # column. @@ -606,38 +630,42 @@ upsert uniq │ └── CASE WHEN uniq.k:12 IS NULL THEN column11:11 ELSE uniq.y:16 END [as=upsert_y:23] └── unique-checks ├── unique-checks-item: uniq(w) - │ └── semi-join (hash) - │ ├── columns: k:30 v:31 w:32!null x:33 y:34 - │ ├── with-scan &1 - │ │ ├── columns: k:30 v:31 w:32!null x:33 y:34 - │ │ └── mapping: - │ │ ├── upsert_k:19 => k:30 - │ │ ├── upsert_v:20 => v:31 - │ │ ├── upsert_w:21 => w:32 - │ │ ├── upsert_x:22 => x:33 - │ │ └── upsert_y:23 => y:34 - │ ├── scan uniq - │ │ └── columns: uniq.k:24!null uniq.v:25 uniq.w:26 uniq.x:27 uniq.y:28 - │ └── filters - │ ├── w:32 = uniq.w:26 - │ └── k:30 != uniq.k:24 + │ └── project + │ ├── columns: w:32!null + │ └── semi-join (hash) + │ ├── columns: k:30 v:31 w:32!null x:33 y:34 + │ ├── with-scan &1 + │ │ ├── columns: k:30 v:31 w:32!null x:33 y:34 + │ │ └── mapping: + │ │ ├── upsert_k:19 => k:30 + │ │ ├── upsert_v:20 => v:31 + │ │ ├── upsert_w:21 => w:32 + │ │ ├── upsert_x:22 => x:33 + │ │ └── upsert_y:23 => y:34 + │ ├── scan uniq + │ │ └── columns: uniq.k:24!null uniq.v:25 uniq.w:26 uniq.x:27 uniq.y:28 + │ └── filters + │ ├── w:32 = uniq.w:26 + │ └── k:30 != uniq.k:24 └── unique-checks-item: uniq(x,y) - └── semi-join (hash) - ├── columns: k:41 v:42 w:43!null x:44 y:45 - ├── with-scan &1 - │ ├── columns: k:41 v:42 w:43!null x:44 y:45 - │ └── mapping: - │ ├── upsert_k:19 => k:41 - │ ├── upsert_v:20 => v:42 - │ ├── upsert_w:21 => w:43 - │ ├── upsert_x:22 => x:44 - │ └── upsert_y:23 => y:45 - ├── scan uniq - │ └── columns: uniq.k:35!null uniq.v:36 uniq.w:37 uniq.x:38 uniq.y:39 - └── filters - ├── x:44 = uniq.x:38 - ├── y:45 = uniq.y:39 - └── k:41 != uniq.k:35 + └── project + ├── columns: x:44 y:45 + └── semi-join (hash) + ├── columns: k:41 v:42 w:43!null x:44 y:45 + ├── with-scan &1 + │ ├── columns: k:41 v:42 w:43!null x:44 y:45 + │ └── mapping: + │ ├── upsert_k:19 => k:41 + │ ├── upsert_v:20 => v:42 + │ ├── upsert_w:21 => w:43 + │ ├── upsert_x:22 => x:44 + │ └── upsert_y:23 => y:45 + ├── scan uniq + │ └── columns: uniq.k:35!null uniq.v:36 uniq.w:37 uniq.x:38 uniq.y:39 + └── filters + ├── x:44 = uniq.x:38 + ├── y:45 = uniq.y:39 + └── k:41 != uniq.k:35 # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # columns. @@ -692,21 +720,23 @@ upsert uniq │ └── CASE WHEN uniq.k:12 IS NULL THEN column5:11 ELSE uniq.y:16 END [as=upsert_y:23] └── unique-checks └── unique-checks-item: uniq(w) - └── semi-join (hash) - ├── columns: k:30 v:31!null w:32 x:33 y:34 - ├── with-scan &1 - │ ├── columns: k:30 v:31!null w:32 x:33 y:34 - │ └── mapping: - │ ├── upsert_k:19 => k:30 - │ ├── upsert_v:20 => v:31 - │ ├── upsert_w:21 => w:32 - │ ├── upsert_x:22 => x:33 - │ └── upsert_y:23 => y:34 - ├── scan uniq - │ └── columns: uniq.k:24!null uniq.v:25 uniq.w:26 uniq.x:27 uniq.y:28 - └── filters - ├── w:32 = uniq.w:26 - └── k:30 != uniq.k:24 + └── project + ├── columns: w:32 + └── semi-join (hash) + ├── columns: k:30 v:31!null w:32 x:33 y:34 + ├── with-scan &1 + │ ├── columns: k:30 v:31!null w:32 x:33 y:34 + │ └── mapping: + │ ├── upsert_k:19 => k:30 + │ ├── upsert_v:20 => v:31 + │ ├── upsert_w:21 => w:32 + │ ├── upsert_x:22 => x:33 + │ └── upsert_y:23 => y:34 + ├── scan uniq + │ └── columns: uniq.k:24!null uniq.v:25 uniq.w:26 uniq.x:27 uniq.y:28 + └── filters + ├── w:32 = uniq.w:26 + └── k:30 != uniq.k:24 # Cannot conflict on a subset of columns in a unique constraint. build @@ -754,52 +784,58 @@ upsert uniq_overlaps_pk │ └── (2, 2, 2, 2) └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ ├── with-scan &1 - │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null - │ │ └── mapping: - │ │ ├── column1:6 => a:15 - │ │ ├── column2:7 => b:16 - │ │ ├── column3:8 => c:17 - │ │ └── column4:9 => d:18 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:10!null uniq_overlaps_pk.b:11!null uniq_overlaps_pk.c:12 uniq_overlaps_pk.d:13 - │ └── filters - │ ├── b:16 = uniq_overlaps_pk.b:11 - │ ├── c:17 = uniq_overlaps_pk.c:12 - │ └── a:15 != uniq_overlaps_pk.a:10 + │ └── project + │ ├── columns: b:16!null c:17!null + │ └── semi-join (hash) + │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ ├── with-scan &1 + │ │ ├── columns: a:15!null b:16!null c:17!null d:18!null + │ │ └── mapping: + │ │ ├── column1:6 => a:15 + │ │ ├── column2:7 => b:16 + │ │ ├── column3:8 => c:17 + │ │ └── column4:9 => d:18 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:10!null uniq_overlaps_pk.b:11!null uniq_overlaps_pk.c:12 uniq_overlaps_pk.d:13 + │ └── filters + │ ├── b:16 = uniq_overlaps_pk.b:11 + │ ├── c:17 = uniq_overlaps_pk.c:12 + │ └── a:15 != uniq_overlaps_pk.a:10 ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ ├── with-scan &1 - │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null - │ │ └── mapping: - │ │ ├── column1:6 => a:24 - │ │ ├── column2:7 => b:25 - │ │ ├── column3:8 => c:26 - │ │ └── column4:9 => d:27 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:19!null uniq_overlaps_pk.b:20!null uniq_overlaps_pk.c:21 uniq_overlaps_pk.d:22 - │ └── filters - │ ├── a:24 = uniq_overlaps_pk.a:19 - │ └── b:25 != uniq_overlaps_pk.b:20 + │ └── project + │ ├── columns: a:24!null + │ └── semi-join (hash) + │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ ├── with-scan &1 + │ │ ├── columns: a:24!null b:25!null c:26!null d:27!null + │ │ └── mapping: + │ │ ├── column1:6 => a:24 + │ │ ├── column2:7 => b:25 + │ │ ├── column3:8 => c:26 + │ │ └── column4:9 => d:27 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:19!null uniq_overlaps_pk.b:20!null uniq_overlaps_pk.c:21 uniq_overlaps_pk.d:22 + │ └── filters + │ ├── a:24 = uniq_overlaps_pk.a:19 + │ └── b:25 != uniq_overlaps_pk.b:20 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:33!null b:34!null c:35!null d:36!null - ├── with-scan &1 - │ ├── columns: a:33!null b:34!null c:35!null d:36!null - │ └── mapping: - │ ├── column1:6 => a:33 - │ ├── column2:7 => b:34 - │ ├── column3:8 => c:35 - │ └── column4:9 => d:36 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:28!null uniq_overlaps_pk.b:29!null uniq_overlaps_pk.c:30 uniq_overlaps_pk.d:31 - └── filters - ├── c:35 = uniq_overlaps_pk.c:30 - ├── d:36 = uniq_overlaps_pk.d:31 - └── (a:33 != uniq_overlaps_pk.a:28) OR (b:34 != uniq_overlaps_pk.b:29) + └── project + ├── columns: c:35!null d:36!null + └── semi-join (hash) + ├── columns: a:33!null b:34!null c:35!null d:36!null + ├── with-scan &1 + │ ├── columns: a:33!null b:34!null c:35!null d:36!null + │ └── mapping: + │ ├── column1:6 => a:33 + │ ├── column2:7 => b:34 + │ ├── column3:8 => c:35 + │ └── column4:9 => d:36 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:28!null uniq_overlaps_pk.b:29!null uniq_overlaps_pk.c:30 uniq_overlaps_pk.d:31 + └── filters + ├── c:35 = uniq_overlaps_pk.c:30 + ├── d:36 = uniq_overlaps_pk.d:31 + └── (a:33 != uniq_overlaps_pk.a:28) OR (b:34 != uniq_overlaps_pk.b:29) # Upsert with non-constant input. # Add inequality filters for the primary key columns that are not part of each @@ -826,36 +862,40 @@ upsert uniq_overlaps_pk │ └── NULL::INT8 [as=column13:13] └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:19 b:20 c:21 d:22 - │ ├── with-scan &1 - │ │ ├── columns: a:19 b:20 c:21 d:22 - │ │ └── mapping: - │ │ ├── k:6 => a:19 - │ │ ├── v:7 => b:20 - │ │ ├── x:9 => c:21 - │ │ └── column13:13 => d:22 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:14!null uniq_overlaps_pk.b:15!null uniq_overlaps_pk.c:16 uniq_overlaps_pk.d:17 - │ └── filters - │ ├── b:20 = uniq_overlaps_pk.b:15 - │ ├── c:21 = uniq_overlaps_pk.c:16 - │ └── a:19 != uniq_overlaps_pk.a:14 + │ └── project + │ ├── columns: b:20 c:21 + │ └── semi-join (hash) + │ ├── columns: a:19 b:20 c:21 d:22 + │ ├── with-scan &1 + │ │ ├── columns: a:19 b:20 c:21 d:22 + │ │ └── mapping: + │ │ ├── k:6 => a:19 + │ │ ├── v:7 => b:20 + │ │ ├── x:9 => c:21 + │ │ └── column13:13 => d:22 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:14!null uniq_overlaps_pk.b:15!null uniq_overlaps_pk.c:16 uniq_overlaps_pk.d:17 + │ └── filters + │ ├── b:20 = uniq_overlaps_pk.b:15 + │ ├── c:21 = uniq_overlaps_pk.c:16 + │ └── a:19 != uniq_overlaps_pk.a:14 └── unique-checks-item: uniq_overlaps_pk(a) - └── semi-join (hash) - ├── columns: a:28 b:29 c:30 d:31 - ├── with-scan &1 - │ ├── columns: a:28 b:29 c:30 d:31 - │ └── mapping: - │ ├── k:6 => a:28 - │ ├── v:7 => b:29 - │ ├── x:9 => c:30 - │ └── column13:13 => d:31 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:23!null uniq_overlaps_pk.b:24!null uniq_overlaps_pk.c:25 uniq_overlaps_pk.d:26 - └── filters - ├── a:28 = uniq_overlaps_pk.a:23 - └── b:29 != uniq_overlaps_pk.b:24 + └── project + ├── columns: a:28 + └── semi-join (hash) + ├── columns: a:28 b:29 c:30 d:31 + ├── with-scan &1 + │ ├── columns: a:28 b:29 c:30 d:31 + │ └── mapping: + │ ├── k:6 => a:28 + │ ├── v:7 => b:29 + │ ├── x:9 => c:30 + │ └── column13:13 => d:31 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:23!null uniq_overlaps_pk.b:24!null uniq_overlaps_pk.c:25 uniq_overlaps_pk.d:26 + └── filters + ├── a:28 = uniq_overlaps_pk.a:23 + └── b:29 != uniq_overlaps_pk.b:24 # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # column. @@ -908,52 +948,58 @@ upsert uniq_overlaps_pk │ └── CASE WHEN uniq_overlaps_pk.a:10 IS NULL THEN column4:9 ELSE uniq_overlaps_pk.d:13 END [as=upsert_d:19] └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:25!null b:26 c:27 d:28 - │ ├── with-scan &1 - │ │ ├── columns: a:25!null b:26 c:27 d:28 - │ │ └── mapping: - │ │ ├── upsert_a:16 => a:25 - │ │ ├── upsert_b:17 => b:26 - │ │ ├── upsert_c:18 => c:27 - │ │ └── upsert_d:19 => d:28 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:20!null uniq_overlaps_pk.b:21!null uniq_overlaps_pk.c:22 uniq_overlaps_pk.d:23 - │ └── filters - │ ├── b:26 = uniq_overlaps_pk.b:21 - │ ├── c:27 = uniq_overlaps_pk.c:22 - │ └── a:25 != uniq_overlaps_pk.a:20 + │ └── project + │ ├── columns: b:26 c:27 + │ └── semi-join (hash) + │ ├── columns: a:25!null b:26 c:27 d:28 + │ ├── with-scan &1 + │ │ ├── columns: a:25!null b:26 c:27 d:28 + │ │ └── mapping: + │ │ ├── upsert_a:16 => a:25 + │ │ ├── upsert_b:17 => b:26 + │ │ ├── upsert_c:18 => c:27 + │ │ └── upsert_d:19 => d:28 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:20!null uniq_overlaps_pk.b:21!null uniq_overlaps_pk.c:22 uniq_overlaps_pk.d:23 + │ └── filters + │ ├── b:26 = uniq_overlaps_pk.b:21 + │ ├── c:27 = uniq_overlaps_pk.c:22 + │ └── a:25 != uniq_overlaps_pk.a:20 ├── unique-checks-item: uniq_overlaps_pk(a) - │ └── semi-join (hash) - │ ├── columns: a:34!null b:35 c:36 d:37 - │ ├── with-scan &1 - │ │ ├── columns: a:34!null b:35 c:36 d:37 - │ │ └── mapping: - │ │ ├── upsert_a:16 => a:34 - │ │ ├── upsert_b:17 => b:35 - │ │ ├── upsert_c:18 => c:36 - │ │ └── upsert_d:19 => d:37 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:29!null uniq_overlaps_pk.b:30!null uniq_overlaps_pk.c:31 uniq_overlaps_pk.d:32 - │ └── filters - │ ├── a:34 = uniq_overlaps_pk.a:29 - │ └── b:35 != uniq_overlaps_pk.b:30 + │ └── project + │ ├── columns: a:34!null + │ └── semi-join (hash) + │ ├── columns: a:34!null b:35 c:36 d:37 + │ ├── with-scan &1 + │ │ ├── columns: a:34!null b:35 c:36 d:37 + │ │ └── mapping: + │ │ ├── upsert_a:16 => a:34 + │ │ ├── upsert_b:17 => b:35 + │ │ ├── upsert_c:18 => c:36 + │ │ └── upsert_d:19 => d:37 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:29!null uniq_overlaps_pk.b:30!null uniq_overlaps_pk.c:31 uniq_overlaps_pk.d:32 + │ └── filters + │ ├── a:34 = uniq_overlaps_pk.a:29 + │ └── b:35 != uniq_overlaps_pk.b:30 └── unique-checks-item: uniq_overlaps_pk(c,d) - └── semi-join (hash) - ├── columns: a:43!null b:44 c:45 d:46 - ├── with-scan &1 - │ ├── columns: a:43!null b:44 c:45 d:46 - │ └── mapping: - │ ├── upsert_a:16 => a:43 - │ ├── upsert_b:17 => b:44 - │ ├── upsert_c:18 => c:45 - │ └── upsert_d:19 => d:46 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:38!null uniq_overlaps_pk.b:39!null uniq_overlaps_pk.c:40 uniq_overlaps_pk.d:41 - └── filters - ├── c:45 = uniq_overlaps_pk.c:40 - ├── d:46 = uniq_overlaps_pk.d:41 - └── (a:43 != uniq_overlaps_pk.a:38) OR (b:44 != uniq_overlaps_pk.b:39) + └── project + ├── columns: c:45 d:46 + └── semi-join (hash) + ├── columns: a:43!null b:44 c:45 d:46 + ├── with-scan &1 + │ ├── columns: a:43!null b:44 c:45 d:46 + │ └── mapping: + │ ├── upsert_a:16 => a:43 + │ ├── upsert_b:17 => b:44 + │ ├── upsert_c:18 => c:45 + │ └── upsert_d:19 => d:46 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:38!null uniq_overlaps_pk.b:39!null uniq_overlaps_pk.c:40 uniq_overlaps_pk.d:41 + └── filters + ├── c:45 = uniq_overlaps_pk.c:40 + ├── d:46 = uniq_overlaps_pk.d:41 + └── (a:43 != uniq_overlaps_pk.a:38) OR (b:44 != uniq_overlaps_pk.b:39) # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # columns. @@ -1004,36 +1050,40 @@ upsert uniq_overlaps_pk │ └── CASE WHEN uniq_overlaps_pk.a:10 IS NULL THEN column4:9 ELSE uniq_overlaps_pk.d:13 END [as=upsert_d:19] └── unique-checks ├── unique-checks-item: uniq_overlaps_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:25 b:26!null c:27 d:28 - │ ├── with-scan &1 - │ │ ├── columns: a:25 b:26!null c:27 d:28 - │ │ └── mapping: - │ │ ├── upsert_a:16 => a:25 - │ │ ├── upsert_b:17 => b:26 - │ │ ├── upsert_c:18 => c:27 - │ │ └── upsert_d:19 => d:28 - │ ├── scan uniq_overlaps_pk - │ │ └── columns: uniq_overlaps_pk.a:20!null uniq_overlaps_pk.b:21!null uniq_overlaps_pk.c:22 uniq_overlaps_pk.d:23 - │ └── filters - │ ├── b:26 = uniq_overlaps_pk.b:21 - │ ├── c:27 = uniq_overlaps_pk.c:22 - │ └── a:25 != uniq_overlaps_pk.a:20 + │ └── project + │ ├── columns: b:26!null c:27 + │ └── semi-join (hash) + │ ├── columns: a:25 b:26!null c:27 d:28 + │ ├── with-scan &1 + │ │ ├── columns: a:25 b:26!null c:27 d:28 + │ │ └── mapping: + │ │ ├── upsert_a:16 => a:25 + │ │ ├── upsert_b:17 => b:26 + │ │ ├── upsert_c:18 => c:27 + │ │ └── upsert_d:19 => d:28 + │ ├── scan uniq_overlaps_pk + │ │ └── columns: uniq_overlaps_pk.a:20!null uniq_overlaps_pk.b:21!null uniq_overlaps_pk.c:22 uniq_overlaps_pk.d:23 + │ └── filters + │ ├── b:26 = uniq_overlaps_pk.b:21 + │ ├── c:27 = uniq_overlaps_pk.c:22 + │ └── a:25 != uniq_overlaps_pk.a:20 └── unique-checks-item: uniq_overlaps_pk(a) - └── semi-join (hash) - ├── columns: a:34 b:35!null c:36 d:37 - ├── with-scan &1 - │ ├── columns: a:34 b:35!null c:36 d:37 - │ └── mapping: - │ ├── upsert_a:16 => a:34 - │ ├── upsert_b:17 => b:35 - │ ├── upsert_c:18 => c:36 - │ └── upsert_d:19 => d:37 - ├── scan uniq_overlaps_pk - │ └── columns: uniq_overlaps_pk.a:29!null uniq_overlaps_pk.b:30!null uniq_overlaps_pk.c:31 uniq_overlaps_pk.d:32 - └── filters - ├── a:34 = uniq_overlaps_pk.a:29 - └── b:35 != uniq_overlaps_pk.b:30 + └── project + ├── columns: a:34 + └── semi-join (hash) + ├── columns: a:34 b:35!null c:36 d:37 + ├── with-scan &1 + │ ├── columns: a:34 b:35!null c:36 d:37 + │ └── mapping: + │ ├── upsert_a:16 => a:34 + │ ├── upsert_b:17 => b:35 + │ ├── upsert_c:18 => c:36 + │ └── upsert_d:19 => d:37 + ├── scan uniq_overlaps_pk + │ └── columns: uniq_overlaps_pk.a:29!null uniq_overlaps_pk.b:30!null uniq_overlaps_pk.c:31 uniq_overlaps_pk.d:32 + └── filters + ├── a:34 = uniq_overlaps_pk.a:29 + └── b:35 != uniq_overlaps_pk.b:30 exec-ddl CREATE TABLE uniq_hidden_pk ( @@ -1102,56 +1152,62 @@ upsert uniq_hidden_pk │ └── CASE WHEN uniq_hidden_pk.rowid:16 IS NULL THEN column11:11 ELSE uniq_hidden_pk.rowid:16 END [as=upsert_rowid:19] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:26!null b:27!null c:28 d:29!null rowid:30 - │ ├── with-scan &1 - │ │ ├── columns: a:26!null b:27!null c:28 d:29!null rowid:30 - │ │ └── mapping: - │ │ ├── column1:7 => a:26 - │ │ ├── column2:8 => b:27 - │ │ ├── upsert_c:18 => c:28 - │ │ ├── column3:9 => d:29 - │ │ └── upsert_rowid:19 => rowid:30 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:20 uniq_hidden_pk.b:21 uniq_hidden_pk.c:22 uniq_hidden_pk.d:23 uniq_hidden_pk.rowid:24!null - │ └── filters - │ ├── b:27 = uniq_hidden_pk.b:21 - │ ├── c:28 = uniq_hidden_pk.c:22 - │ └── rowid:30 != uniq_hidden_pk.rowid:24 + │ └── project + │ ├── columns: b:27!null c:28 + │ └── semi-join (hash) + │ ├── columns: a:26!null b:27!null c:28 d:29!null rowid:30 + │ ├── with-scan &1 + │ │ ├── columns: a:26!null b:27!null c:28 d:29!null rowid:30 + │ │ └── mapping: + │ │ ├── column1:7 => a:26 + │ │ ├── column2:8 => b:27 + │ │ ├── upsert_c:18 => c:28 + │ │ ├── column3:9 => d:29 + │ │ └── upsert_rowid:19 => rowid:30 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:20 uniq_hidden_pk.b:21 uniq_hidden_pk.c:22 uniq_hidden_pk.d:23 uniq_hidden_pk.rowid:24!null + │ └── filters + │ ├── b:27 = uniq_hidden_pk.b:21 + │ ├── c:28 = uniq_hidden_pk.c:22 + │ └── rowid:30 != uniq_hidden_pk.rowid:24 ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:37!null b:38!null c:39 d:40!null rowid:41 - │ ├── with-scan &1 - │ │ ├── columns: a:37!null b:38!null c:39 d:40!null rowid:41 - │ │ └── mapping: - │ │ ├── column1:7 => a:37 - │ │ ├── column2:8 => b:38 - │ │ ├── upsert_c:18 => c:39 - │ │ ├── column3:9 => d:40 - │ │ └── upsert_rowid:19 => rowid:41 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:31 uniq_hidden_pk.b:32 uniq_hidden_pk.c:33 uniq_hidden_pk.d:34 uniq_hidden_pk.rowid:35!null - │ └── filters - │ ├── a:37 = uniq_hidden_pk.a:31 - │ ├── b:38 = uniq_hidden_pk.b:32 - │ ├── d:40 = uniq_hidden_pk.d:34 - │ └── rowid:41 != uniq_hidden_pk.rowid:35 + │ └── project + │ ├── columns: a:37!null b:38!null d:40!null + │ └── semi-join (hash) + │ ├── columns: a:37!null b:38!null c:39 d:40!null rowid:41 + │ ├── with-scan &1 + │ │ ├── columns: a:37!null b:38!null c:39 d:40!null rowid:41 + │ │ └── mapping: + │ │ ├── column1:7 => a:37 + │ │ ├── column2:8 => b:38 + │ │ ├── upsert_c:18 => c:39 + │ │ ├── column3:9 => d:40 + │ │ └── upsert_rowid:19 => rowid:41 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:31 uniq_hidden_pk.b:32 uniq_hidden_pk.c:33 uniq_hidden_pk.d:34 uniq_hidden_pk.rowid:35!null + │ └── filters + │ ├── a:37 = uniq_hidden_pk.a:31 + │ ├── b:38 = uniq_hidden_pk.b:32 + │ ├── d:40 = uniq_hidden_pk.d:34 + │ └── rowid:41 != uniq_hidden_pk.rowid:35 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:48!null b:49!null c:50 d:51!null rowid:52 - ├── with-scan &1 - │ ├── columns: a:48!null b:49!null c:50 d:51!null rowid:52 - │ └── mapping: - │ ├── column1:7 => a:48 - │ ├── column2:8 => b:49 - │ ├── upsert_c:18 => c:50 - │ ├── column3:9 => d:51 - │ └── upsert_rowid:19 => rowid:52 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:42 uniq_hidden_pk.b:43 uniq_hidden_pk.c:44 uniq_hidden_pk.d:45 uniq_hidden_pk.rowid:46!null - └── filters - ├── a:48 = uniq_hidden_pk.a:42 - └── rowid:52 != uniq_hidden_pk.rowid:46 + └── project + ├── columns: a:48!null + └── semi-join (hash) + ├── columns: a:48!null b:49!null c:50 d:51!null rowid:52 + ├── with-scan &1 + │ ├── columns: a:48!null b:49!null c:50 d:51!null rowid:52 + │ └── mapping: + │ ├── column1:7 => a:48 + │ ├── column2:8 => b:49 + │ ├── upsert_c:18 => c:50 + │ ├── column3:9 => d:51 + │ └── upsert_rowid:19 => rowid:52 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:42 uniq_hidden_pk.b:43 uniq_hidden_pk.c:44 uniq_hidden_pk.d:45 uniq_hidden_pk.rowid:46!null + └── filters + ├── a:48 = uniq_hidden_pk.a:42 + └── rowid:52 != uniq_hidden_pk.rowid:46 # Upsert with non-constant input. # Add inequality filters for the hidden primary key column. @@ -1177,56 +1233,62 @@ upsert uniq_hidden_pk │ └── unique_rowid() [as=column14:14] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 - │ ├── with-scan &1 - │ │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 - │ │ └── mapping: - │ │ ├── k:7 => a:21 - │ │ ├── v:8 => b:22 - │ │ ├── x:10 => c:23 - │ │ ├── y:11 => d:24 - │ │ └── column14:14 => rowid:25 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:15 uniq_hidden_pk.b:16 uniq_hidden_pk.c:17 uniq_hidden_pk.d:18 uniq_hidden_pk.rowid:19!null - │ └── filters - │ ├── b:22 = uniq_hidden_pk.b:16 - │ ├── c:23 = uniq_hidden_pk.c:17 - │ └── rowid:25 != uniq_hidden_pk.rowid:19 + │ └── project + │ ├── columns: b:22 c:23 + │ └── semi-join (hash) + │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 + │ ├── with-scan &1 + │ │ ├── columns: a:21 b:22 c:23 d:24 rowid:25 + │ │ └── mapping: + │ │ ├── k:7 => a:21 + │ │ ├── v:8 => b:22 + │ │ ├── x:10 => c:23 + │ │ ├── y:11 => d:24 + │ │ └── column14:14 => rowid:25 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:15 uniq_hidden_pk.b:16 uniq_hidden_pk.c:17 uniq_hidden_pk.d:18 uniq_hidden_pk.rowid:19!null + │ └── filters + │ ├── b:22 = uniq_hidden_pk.b:16 + │ ├── c:23 = uniq_hidden_pk.c:17 + │ └── rowid:25 != uniq_hidden_pk.rowid:19 ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 - │ ├── with-scan &1 - │ │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 - │ │ └── mapping: - │ │ ├── k:7 => a:32 - │ │ ├── v:8 => b:33 - │ │ ├── x:10 => c:34 - │ │ ├── y:11 => d:35 - │ │ └── column14:14 => rowid:36 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:26 uniq_hidden_pk.b:27 uniq_hidden_pk.c:28 uniq_hidden_pk.d:29 uniq_hidden_pk.rowid:30!null - │ └── filters - │ ├── a:32 = uniq_hidden_pk.a:26 - │ ├── b:33 = uniq_hidden_pk.b:27 - │ ├── d:35 = uniq_hidden_pk.d:29 - │ └── rowid:36 != uniq_hidden_pk.rowid:30 + │ └── project + │ ├── columns: a:32 b:33 d:35 + │ └── semi-join (hash) + │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 + │ ├── with-scan &1 + │ │ ├── columns: a:32 b:33 c:34 d:35 rowid:36 + │ │ └── mapping: + │ │ ├── k:7 => a:32 + │ │ ├── v:8 => b:33 + │ │ ├── x:10 => c:34 + │ │ ├── y:11 => d:35 + │ │ └── column14:14 => rowid:36 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:26 uniq_hidden_pk.b:27 uniq_hidden_pk.c:28 uniq_hidden_pk.d:29 uniq_hidden_pk.rowid:30!null + │ └── filters + │ ├── a:32 = uniq_hidden_pk.a:26 + │ ├── b:33 = uniq_hidden_pk.b:27 + │ ├── d:35 = uniq_hidden_pk.d:29 + │ └── rowid:36 != uniq_hidden_pk.rowid:30 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:43 b:44 c:45 d:46 rowid:47 - ├── with-scan &1 - │ ├── columns: a:43 b:44 c:45 d:46 rowid:47 - │ └── mapping: - │ ├── k:7 => a:43 - │ ├── v:8 => b:44 - │ ├── x:10 => c:45 - │ ├── y:11 => d:46 - │ └── column14:14 => rowid:47 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:37 uniq_hidden_pk.b:38 uniq_hidden_pk.c:39 uniq_hidden_pk.d:40 uniq_hidden_pk.rowid:41!null - └── filters - ├── a:43 = uniq_hidden_pk.a:37 - └── rowid:47 != uniq_hidden_pk.rowid:41 + └── project + ├── columns: a:43 + └── semi-join (hash) + ├── columns: a:43 b:44 c:45 d:46 rowid:47 + ├── with-scan &1 + │ ├── columns: a:43 b:44 c:45 d:46 rowid:47 + │ └── mapping: + │ ├── k:7 => a:43 + │ ├── v:8 => b:44 + │ ├── x:10 => c:45 + │ ├── y:11 => d:46 + │ └── column14:14 => rowid:47 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:37 uniq_hidden_pk.b:38 uniq_hidden_pk.c:39 uniq_hidden_pk.d:40 uniq_hidden_pk.rowid:41!null + └── filters + ├── a:43 = uniq_hidden_pk.a:37 + └── rowid:47 != uniq_hidden_pk.rowid:41 # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # columns. @@ -1284,56 +1346,62 @@ upsert uniq_hidden_pk │ └── CASE WHEN uniq_hidden_pk.rowid:16 IS NULL THEN column11:11 ELSE uniq_hidden_pk.rowid:16 END [as=upsert_rowid:23] └── unique-checks ├── unique-checks-item: uniq_hidden_pk(b,c) - │ └── semi-join (hash) - │ ├── columns: a:30!null b:31 c:32 d:33 rowid:34 - │ ├── with-scan &1 - │ │ ├── columns: a:30!null b:31 c:32 d:33 rowid:34 - │ │ └── mapping: - │ │ ├── upsert_a:19 => a:30 - │ │ ├── upsert_b:20 => b:31 - │ │ ├── upsert_c:21 => c:32 - │ │ ├── upsert_d:22 => d:33 - │ │ └── upsert_rowid:23 => rowid:34 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:24 uniq_hidden_pk.b:25 uniq_hidden_pk.c:26 uniq_hidden_pk.d:27 uniq_hidden_pk.rowid:28!null - │ └── filters - │ ├── b:31 = uniq_hidden_pk.b:25 - │ ├── c:32 = uniq_hidden_pk.c:26 - │ └── rowid:34 != uniq_hidden_pk.rowid:28 + │ └── project + │ ├── columns: b:31 c:32 + │ └── semi-join (hash) + │ ├── columns: a:30!null b:31 c:32 d:33 rowid:34 + │ ├── with-scan &1 + │ │ ├── columns: a:30!null b:31 c:32 d:33 rowid:34 + │ │ └── mapping: + │ │ ├── upsert_a:19 => a:30 + │ │ ├── upsert_b:20 => b:31 + │ │ ├── upsert_c:21 => c:32 + │ │ ├── upsert_d:22 => d:33 + │ │ └── upsert_rowid:23 => rowid:34 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:24 uniq_hidden_pk.b:25 uniq_hidden_pk.c:26 uniq_hidden_pk.d:27 uniq_hidden_pk.rowid:28!null + │ └── filters + │ ├── b:31 = uniq_hidden_pk.b:25 + │ ├── c:32 = uniq_hidden_pk.c:26 + │ └── rowid:34 != uniq_hidden_pk.rowid:28 ├── unique-checks-item: uniq_hidden_pk(a,b,d) - │ └── semi-join (hash) - │ ├── columns: a:41!null b:42 c:43 d:44 rowid:45 - │ ├── with-scan &1 - │ │ ├── columns: a:41!null b:42 c:43 d:44 rowid:45 - │ │ └── mapping: - │ │ ├── upsert_a:19 => a:41 - │ │ ├── upsert_b:20 => b:42 - │ │ ├── upsert_c:21 => c:43 - │ │ ├── upsert_d:22 => d:44 - │ │ └── upsert_rowid:23 => rowid:45 - │ ├── scan uniq_hidden_pk - │ │ └── columns: uniq_hidden_pk.a:35 uniq_hidden_pk.b:36 uniq_hidden_pk.c:37 uniq_hidden_pk.d:38 uniq_hidden_pk.rowid:39!null - │ └── filters - │ ├── a:41 = uniq_hidden_pk.a:35 - │ ├── b:42 = uniq_hidden_pk.b:36 - │ ├── d:44 = uniq_hidden_pk.d:38 - │ └── rowid:45 != uniq_hidden_pk.rowid:39 + │ └── project + │ ├── columns: a:41!null b:42 d:44 + │ └── semi-join (hash) + │ ├── columns: a:41!null b:42 c:43 d:44 rowid:45 + │ ├── with-scan &1 + │ │ ├── columns: a:41!null b:42 c:43 d:44 rowid:45 + │ │ └── mapping: + │ │ ├── upsert_a:19 => a:41 + │ │ ├── upsert_b:20 => b:42 + │ │ ├── upsert_c:21 => c:43 + │ │ ├── upsert_d:22 => d:44 + │ │ └── upsert_rowid:23 => rowid:45 + │ ├── scan uniq_hidden_pk + │ │ └── columns: uniq_hidden_pk.a:35 uniq_hidden_pk.b:36 uniq_hidden_pk.c:37 uniq_hidden_pk.d:38 uniq_hidden_pk.rowid:39!null + │ └── filters + │ ├── a:41 = uniq_hidden_pk.a:35 + │ ├── b:42 = uniq_hidden_pk.b:36 + │ ├── d:44 = uniq_hidden_pk.d:38 + │ └── rowid:45 != uniq_hidden_pk.rowid:39 └── unique-checks-item: uniq_hidden_pk(a) - └── semi-join (hash) - ├── columns: a:52!null b:53 c:54 d:55 rowid:56 - ├── with-scan &1 - │ ├── columns: a:52!null b:53 c:54 d:55 rowid:56 - │ └── mapping: - │ ├── upsert_a:19 => a:52 - │ ├── upsert_b:20 => b:53 - │ ├── upsert_c:21 => c:54 - │ ├── upsert_d:22 => d:55 - │ └── upsert_rowid:23 => rowid:56 - ├── scan uniq_hidden_pk - │ └── columns: uniq_hidden_pk.a:46 uniq_hidden_pk.b:47 uniq_hidden_pk.c:48 uniq_hidden_pk.d:49 uniq_hidden_pk.rowid:50!null - └── filters - ├── a:52 = uniq_hidden_pk.a:46 - └── rowid:56 != uniq_hidden_pk.rowid:50 + └── project + ├── columns: a:52!null + └── semi-join (hash) + ├── columns: a:52!null b:53 c:54 d:55 rowid:56 + ├── with-scan &1 + │ ├── columns: a:52!null b:53 c:54 d:55 rowid:56 + │ └── mapping: + │ ├── upsert_a:19 => a:52 + │ ├── upsert_b:20 => b:53 + │ ├── upsert_c:21 => c:54 + │ ├── upsert_d:22 => d:55 + │ └── upsert_rowid:23 => rowid:56 + ├── scan uniq_hidden_pk + │ └── columns: uniq_hidden_pk.a:46 uniq_hidden_pk.b:47 uniq_hidden_pk.c:48 uniq_hidden_pk.d:49 uniq_hidden_pk.rowid:50!null + └── filters + ├── a:52 = uniq_hidden_pk.a:46 + └── rowid:56 != uniq_hidden_pk.rowid:50 exec-ddl CREATE TABLE uniq_fk_parent ( @@ -1394,18 +1462,20 @@ upsert uniq_fk_parent │ └── CASE WHEN uniq_fk_parent.rowid:7 IS NULL THEN column5:5 ELSE uniq_fk_parent.rowid:7 END [as=upsert_rowid:9] ├── unique-checks │ └── unique-checks-item: uniq_fk_parent(a) - │ └── semi-join (hash) - │ ├── columns: a:13!null rowid:14 - │ ├── with-scan &1 - │ │ ├── columns: a:13!null rowid:14 - │ │ └── mapping: - │ │ ├── column1:4 => a:13 - │ │ └── upsert_rowid:9 => rowid:14 - │ ├── scan uniq_fk_parent - │ │ └── columns: uniq_fk_parent.a:10 uniq_fk_parent.rowid:11!null - │ └── filters - │ ├── a:13 = uniq_fk_parent.a:10 - │ └── rowid:14 != uniq_fk_parent.rowid:11 + │ └── project + │ ├── columns: a:13!null + │ └── semi-join (hash) + │ ├── columns: a:13!null rowid:14 + │ ├── with-scan &1 + │ │ ├── columns: a:13!null rowid:14 + │ │ └── mapping: + │ │ ├── column1:4 => a:13 + │ │ └── upsert_rowid:9 => rowid:14 + │ ├── scan uniq_fk_parent + │ │ └── columns: uniq_fk_parent.a:10 uniq_fk_parent.rowid:11!null + │ └── filters + │ ├── a:13 = uniq_fk_parent.a:10 + │ └── rowid:14 != uniq_fk_parent.rowid:11 └── f-k-checks └── f-k-checks-item: uniq_fk_child(a) -> uniq_fk_parent(a) └── semi-join (hash) @@ -1517,21 +1587,23 @@ upsert t │ └── t.i:6 > 0 [as=partial_index_del1:13] └── unique-checks └── unique-checks-item: t(i) - └── semi-join (hash) - ├── columns: i:17!null rowid:18 - ├── with-scan &1 - │ ├── columns: i:17!null rowid:18 - │ └── mapping: - │ ├── upsert_i:10 => i:17 - │ └── upsert_rowid:11 => rowid:18 - ├── scan t - │ ├── columns: t.i:14 t.rowid:15!null - │ └── partial index predicates - │ └── i1: filters - │ └── t.i:14 > 0 - └── filters - ├── i:17 = t.i:14 - └── rowid:18 != t.rowid:15 + └── project + ├── columns: i:17!null + └── semi-join (hash) + ├── columns: i:17!null rowid:18 + ├── with-scan &1 + │ ├── columns: i:17!null rowid:18 + │ └── mapping: + │ ├── upsert_i:10 => i:17 + │ └── upsert_rowid:11 => rowid:18 + ├── scan t + │ ├── columns: t.i:14 t.rowid:15!null + │ └── partial index predicates + │ └── i1: filters + │ └── t.i:14 > 0 + └── filters + ├── i:17 = t.i:14 + └── rowid:18 != t.rowid:15 exec-ddl CREATE TABLE uniq_partial ( @@ -1559,21 +1631,23 @@ upsert uniq_partial │ └── (2, 2, 2) └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:12!null a:13!null b:14!null - ├── with-scan &1 - │ ├── columns: k:12!null a:13!null b:14!null - │ └── mapping: - │ ├── column1:5 => k:12 - │ ├── column2:6 => a:13 - │ └── column3:7 => b:14 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 - └── filters - ├── a:13 = uniq_partial.a:9 - ├── b:14 > 0 - ├── uniq_partial.b:10 > 0 - └── k:12 != uniq_partial.k:8 + └── project + ├── columns: a:13!null + └── semi-join (hash) + ├── columns: k:12!null a:13!null b:14!null + ├── with-scan &1 + │ ├── columns: k:12!null a:13!null b:14!null + │ └── mapping: + │ ├── column1:5 => k:12 + │ ├── column2:6 => a:13 + │ └── column3:7 => b:14 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:8!null uniq_partial.a:9 uniq_partial.b:10 + └── filters + ├── a:13 = uniq_partial.a:9 + ├── b:14 > 0 + ├── uniq_partial.b:10 > 0 + └── k:12 != uniq_partial.k:8 # No need to plan checks for a since it's always null. build @@ -1607,21 +1681,23 @@ upsert uniq_partial │ └── columns: other.k:5 v:6 w:7!null x:8 y:9 rowid:10!null other.crdb_internal_mvcc_timestamp:11 └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:16 a:17 b:18!null - ├── with-scan &1 - │ ├── columns: k:16 a:17 b:18!null - │ └── mapping: - │ ├── other.k:5 => k:16 - │ ├── v:6 => a:17 - │ └── w:7 => b:18 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 - └── filters - ├── a:17 = uniq_partial.a:13 - ├── b:18 > 0 - ├── uniq_partial.b:14 > 0 - └── k:16 != uniq_partial.k:12 + └── project + ├── columns: a:17 + └── semi-join (hash) + ├── columns: k:16 a:17 b:18!null + ├── with-scan &1 + │ ├── columns: k:16 a:17 b:18!null + │ └── mapping: + │ ├── other.k:5 => k:16 + │ ├── v:6 => a:17 + │ └── w:7 => b:18 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:12!null uniq_partial.a:13 uniq_partial.b:14 + └── filters + ├── a:17 = uniq_partial.a:13 + ├── b:18 > 0 + ├── uniq_partial.b:14 > 0 + └── k:16 != uniq_partial.k:12 # On conflict do update with constant input, conflict on UNIQUE WITHOUT INDEX # column and predicate. @@ -1681,21 +1757,23 @@ upsert uniq_partial │ └── CASE WHEN uniq_partial.k:9 IS NULL THEN column3:7 ELSE uniq_partial.b:11 END [as=upsert_b:16] └── unique-checks └── unique-checks-item: uniq_partial(a) - └── semi-join (hash) - ├── columns: k:21 a:22!null b:23 - ├── with-scan &1 - │ ├── columns: k:21 a:22!null b:23 - │ └── mapping: - │ ├── upsert_k:14 => k:21 - │ ├── upsert_a:15 => a:22 - │ └── upsert_b:16 => b:23 - ├── scan uniq_partial - │ └── columns: uniq_partial.k:17!null uniq_partial.a:18 uniq_partial.b:19 - └── filters - ├── a:22 = uniq_partial.a:18 - ├── b:23 > 0 - ├── uniq_partial.b:19 > 0 - └── k:21 != uniq_partial.k:17 + └── project + ├── columns: a:22!null + └── semi-join (hash) + ├── columns: k:21 a:22!null b:23 + ├── with-scan &1 + │ ├── columns: k:21 a:22!null b:23 + │ └── mapping: + │ ├── upsert_k:14 => k:21 + │ ├── upsert_a:15 => a:22 + │ └── upsert_b:16 => b:23 + ├── scan uniq_partial + │ └── columns: uniq_partial.k:17!null uniq_partial.a:18 uniq_partial.b:19 + └── filters + ├── a:22 = uniq_partial.a:18 + ├── b:23 > 0 + ├── uniq_partial.b:19 > 0 + └── k:21 != uniq_partial.k:17 # Error when there is no arbiter predicate to match the partial unique # constraint predicate. @@ -1758,29 +1836,31 @@ upsert uniq_partial_constraint_and_index │ └── CASE WHEN uniq_partial_constraint_and_index.k:9 IS NULL THEN column3:7 ELSE uniq_partial_constraint_and_index.b:11 END [as=upsert_b:16] └── unique-checks └── unique-checks-item: uniq_partial_constraint_and_index(a) - └── semi-join (hash) - ├── columns: k:22 a:23!null b:24!null - ├── select - │ ├── columns: k:22 a:23!null b:24!null - │ ├── with-scan &1 - │ │ ├── columns: k:22 a:23!null b:24 - │ │ └── mapping: - │ │ ├── upsert_k:14 => k:22 - │ │ ├── upsert_a:15 => a:23 - │ │ └── upsert_b:16 => b:24 - │ └── filters - │ └── b:24 > 10 - ├── select - │ ├── columns: uniq_partial_constraint_and_index.k:18!null uniq_partial_constraint_and_index.a:19 uniq_partial_constraint_and_index.b:20!null - │ ├── scan uniq_partial_constraint_and_index - │ │ ├── columns: uniq_partial_constraint_and_index.k:18!null uniq_partial_constraint_and_index.a:19 uniq_partial_constraint_and_index.b:20 - │ │ └── partial index predicates - │ │ └── secondary: filters (true) - │ └── filters - │ └── uniq_partial_constraint_and_index.b:20 > 10 - └── filters - ├── a:23 = uniq_partial_constraint_and_index.a:19 - └── k:22 != uniq_partial_constraint_and_index.k:18 + └── project + ├── columns: a:23!null + └── semi-join (hash) + ├── columns: k:22 a:23!null b:24!null + ├── select + │ ├── columns: k:22 a:23!null b:24!null + │ ├── with-scan &1 + │ │ ├── columns: k:22 a:23!null b:24 + │ │ └── mapping: + │ │ ├── upsert_k:14 => k:22 + │ │ ├── upsert_a:15 => a:23 + │ │ └── upsert_b:16 => b:24 + │ └── filters + │ └── b:24 > 10 + ├── select + │ ├── columns: uniq_partial_constraint_and_index.k:18!null uniq_partial_constraint_and_index.a:19 uniq_partial_constraint_and_index.b:20!null + │ ├── scan uniq_partial_constraint_and_index + │ │ ├── columns: uniq_partial_constraint_and_index.k:18!null uniq_partial_constraint_and_index.a:19 uniq_partial_constraint_and_index.b:20 + │ │ └── partial index predicates + │ │ └── secondary: filters (true) + │ └── filters + │ └── uniq_partial_constraint_and_index.b:20 > 10 + └── filters + ├── a:23 = uniq_partial_constraint_and_index.a:19 + └── k:22 != uniq_partial_constraint_and_index.k:18 exec-ddl CREATE TABLE uniq_constraint_and_partial_index ( @@ -1814,49 +1894,46 @@ upsert uniq_constraint_and_partial_index ├── partial index del columns: partial_index_del1:18 ├── input binding: &1 ├── project - │ ├── columns: partial_index_put1:17 partial_index_del1:18 column1:5!null column2:6!null column3:7!null uniq_constraint_and_partial_index.k:9 uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 upsert_k:14 upsert_a:15!null upsert_b:16 - │ ├── project - │ │ ├── columns: upsert_k:14 upsert_a:15!null upsert_b:16 column1:5!null column2:6!null column3:7!null uniq_constraint_and_partial_index.k:9 uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 - │ │ ├── left-join (cross) - │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null uniq_constraint_and_partial_index.k:9 uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 - │ │ │ ├── values - │ │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null - │ │ │ │ └── (1, 1, 1) - │ │ │ ├── select - │ │ │ │ ├── columns: uniq_constraint_and_partial_index.k:9!null uniq_constraint_and_partial_index.a:10!null uniq_constraint_and_partial_index.b:11 - │ │ │ │ ├── scan uniq_constraint_and_partial_index - │ │ │ │ │ ├── columns: uniq_constraint_and_partial_index.k:9!null uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 - │ │ │ │ │ └── partial index predicates - │ │ │ │ │ └── secondary: filters - │ │ │ │ │ └── uniq_constraint_and_partial_index.b:11 > 0 - │ │ │ │ └── filters - │ │ │ │ └── uniq_constraint_and_partial_index.a:10 = 1 - │ │ │ └── filters (true) - │ │ └── projections - │ │ ├── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column1:5 ELSE uniq_constraint_and_partial_index.k:9 END [as=upsert_k:14] - │ │ ├── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column2:6 ELSE 10 END [as=upsert_a:15] - │ │ └── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column3:7 ELSE uniq_constraint_and_partial_index.b:11 END [as=upsert_b:16] + │ ├── columns: partial_index_put1:17 partial_index_del1:18 upsert_k:14 upsert_a:15!null column1:5!null column2:6!null column3:7!null uniq_constraint_and_partial_index.k:9 uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 + │ ├── left-join (cross) + │ │ ├── columns: column1:5!null column2:6!null column3:7!null uniq_constraint_and_partial_index.k:9 uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 + │ │ ├── values + │ │ │ ├── columns: column1:5!null column2:6!null column3:7!null + │ │ │ └── (1, 1, 1) + │ │ ├── select + │ │ │ ├── columns: uniq_constraint_and_partial_index.k:9!null uniq_constraint_and_partial_index.a:10!null uniq_constraint_and_partial_index.b:11 + │ │ │ ├── scan uniq_constraint_and_partial_index + │ │ │ │ ├── columns: uniq_constraint_and_partial_index.k:9!null uniq_constraint_and_partial_index.a:10 uniq_constraint_and_partial_index.b:11 + │ │ │ │ └── partial index predicates + │ │ │ │ └── secondary: filters + │ │ │ │ └── uniq_constraint_and_partial_index.b:11 > 0 + │ │ │ └── filters + │ │ │ └── uniq_constraint_and_partial_index.a:10 = 1 + │ │ └── filters (true) │ └── projections - │ ├── upsert_b:16 > 0 [as=partial_index_put1:17] - │ └── uniq_constraint_and_partial_index.b:11 > 0 [as=partial_index_del1:18] + │ ├── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column3:7 ELSE uniq_constraint_and_partial_index.b:11 END > 0 [as=partial_index_put1:17] + │ ├── uniq_constraint_and_partial_index.b:11 > 0 [as=partial_index_del1:18] + │ ├── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column1:5 ELSE uniq_constraint_and_partial_index.k:9 END [as=upsert_k:14] + │ └── CASE WHEN uniq_constraint_and_partial_index.k:9 IS NULL THEN column2:6 ELSE 10 END [as=upsert_a:15] └── unique-checks └── unique-checks-item: uniq_constraint_and_partial_index(a) - └── semi-join (hash) - ├── columns: k:23 a:24!null b:25 - ├── with-scan &1 - │ ├── columns: k:23 a:24!null b:25 - │ └── mapping: - │ ├── upsert_k:14 => k:23 - │ ├── upsert_a:15 => a:24 - │ └── upsert_b:16 => b:25 - ├── scan uniq_constraint_and_partial_index - │ ├── columns: uniq_constraint_and_partial_index.k:19!null uniq_constraint_and_partial_index.a:20 - │ └── partial index predicates - │ └── secondary: filters - │ └── uniq_constraint_and_partial_index.b:21 > 0 - └── filters - ├── a:24 = uniq_constraint_and_partial_index.a:20 - └── k:23 != uniq_constraint_and_partial_index.k:19 + └── project + ├── columns: a:24!null + └── semi-join (hash) + ├── columns: k:23 a:24!null + ├── with-scan &1 + │ ├── columns: k:23 a:24!null + │ └── mapping: + │ ├── upsert_k:14 => k:23 + │ └── upsert_a:15 => a:24 + ├── scan uniq_constraint_and_partial_index + │ ├── columns: uniq_constraint_and_partial_index.k:19!null uniq_constraint_and_partial_index.a:20 + │ └── partial index predicates + │ └── secondary: filters + │ └── uniq_constraint_and_partial_index.b:21 > 0 + └── filters + ├── a:24 = uniq_constraint_and_partial_index.a:20 + └── k:23 != uniq_constraint_and_partial_index.k:19 exec-ddl CREATE TABLE uniq_partial_constraint_and_partial_index ( @@ -1970,33 +2047,35 @@ upsert uniq_computed_pk │ └── CASE WHEN uniq_computed_pk.c_i_expr:17 IS NULL THEN column12:12 ELSE uniq_computed_pk.c_i_expr:17 END [as=upsert_c_i_expr:24] └── unique-checks └── unique-checks-item: uniq_computed_pk(d) - └── semi-join (hash) - ├── columns: i:49 s:50!null d:51!null c_i_expr:52 c_s:53!null c_d:54!null c_d_expr:55!null - ├── with-scan &1 - │ ├── columns: i:49 s:50!null d:51!null c_i_expr:52 c_s:53!null c_d:54!null c_d_expr:55!null - │ └── mapping: - │ ├── upsert_i:23 => i:49 - │ ├── column2:10 => s:50 - │ ├── column3:11 => d:51 - │ ├── upsert_c_i_expr:24 => c_i_expr:52 - │ ├── column2:10 => c_s:53 - │ ├── column3:11 => c_d:54 - │ └── column13:13 => c_d_expr:55 - ├── project - │ ├── columns: uniq_computed_pk.c_s:45 uniq_computed_pk.i:41!null uniq_computed_pk.s:42 uniq_computed_pk.d:43 uniq_computed_pk.c_i_expr:44!null uniq_computed_pk.c_d:46 uniq_computed_pk.c_d_expr:47 - │ ├── scan uniq_computed_pk - │ │ ├── columns: uniq_computed_pk.i:41!null uniq_computed_pk.s:42 uniq_computed_pk.d:43 uniq_computed_pk.c_i_expr:44!null uniq_computed_pk.c_d:46 uniq_computed_pk.c_d_expr:47 - │ │ └── computed column expressions - │ │ ├── uniq_computed_pk.c_i_expr:44 - │ │ │ └── CASE WHEN uniq_computed_pk.i:41 < 0 THEN 'foo' ELSE 'bar' END - │ │ ├── uniq_computed_pk.c_s:45 - │ │ │ └── uniq_computed_pk.s:42 - │ │ ├── uniq_computed_pk.c_d:46 - │ │ │ └── uniq_computed_pk.d:43 - │ │ └── uniq_computed_pk.c_d_expr:47 - │ │ └── uniq_computed_pk.d:43::STRING - │ └── projections - │ └── uniq_computed_pk.s:42 [as=uniq_computed_pk.c_s:45] - └── filters - ├── d:51 = uniq_computed_pk.d:43 - └── (i:49 != uniq_computed_pk.i:41) OR (c_i_expr:52 != uniq_computed_pk.c_i_expr:44) + └── project + ├── columns: d:51!null + └── semi-join (hash) + ├── columns: i:49 s:50!null d:51!null c_i_expr:52 c_s:53!null c_d:54!null c_d_expr:55!null + ├── with-scan &1 + │ ├── columns: i:49 s:50!null d:51!null c_i_expr:52 c_s:53!null c_d:54!null c_d_expr:55!null + │ └── mapping: + │ ├── upsert_i:23 => i:49 + │ ├── column2:10 => s:50 + │ ├── column3:11 => d:51 + │ ├── upsert_c_i_expr:24 => c_i_expr:52 + │ ├── column2:10 => c_s:53 + │ ├── column3:11 => c_d:54 + │ └── column13:13 => c_d_expr:55 + ├── project + │ ├── columns: uniq_computed_pk.c_s:45 uniq_computed_pk.i:41!null uniq_computed_pk.s:42 uniq_computed_pk.d:43 uniq_computed_pk.c_i_expr:44!null uniq_computed_pk.c_d:46 uniq_computed_pk.c_d_expr:47 + │ ├── scan uniq_computed_pk + │ │ ├── columns: uniq_computed_pk.i:41!null uniq_computed_pk.s:42 uniq_computed_pk.d:43 uniq_computed_pk.c_i_expr:44!null uniq_computed_pk.c_d:46 uniq_computed_pk.c_d_expr:47 + │ │ └── computed column expressions + │ │ ├── uniq_computed_pk.c_i_expr:44 + │ │ │ └── CASE WHEN uniq_computed_pk.i:41 < 0 THEN 'foo' ELSE 'bar' END + │ │ ├── uniq_computed_pk.c_s:45 + │ │ │ └── uniq_computed_pk.s:42 + │ │ ├── uniq_computed_pk.c_d:46 + │ │ │ └── uniq_computed_pk.d:43 + │ │ └── uniq_computed_pk.c_d_expr:47 + │ │ └── uniq_computed_pk.d:43::STRING + │ └── projections + │ └── uniq_computed_pk.s:42 [as=uniq_computed_pk.c_s:45] + └── filters + ├── d:51 = uniq_computed_pk.d:43 + └── (i:49 != uniq_computed_pk.i:41) OR (c_i_expr:52 != uniq_computed_pk.c_i_expr:44) From badefde3bf4fe9f2893734568513075fc6e23de0 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sun, 28 Feb 2021 21:59:49 +0100 Subject: [PATCH 5/9] server: fix node decommissioning itself Previously, if a node was asked to decommission itself, the decommissioning process would sometimes hang or fail since the node would become decommissioned and lose RPC access to the rest of the cluster while it was building the response. This patch returns an empty response from the `adminServer.Decommission()` call when setting the final `DECOMMISSIONED` status, thereby avoiding further use of cluster RPC after decommissioning itself. It also defers self-decommissioning until the end if multiple nodes are being decommissioned. This change is backwards-compatible with old CLI versions, which will simply output the now-empty status result set before completing with "No more data reported on target nodes". The CLI has been updated to simply omit the empty response. Release justification: bug fixes and low-risk updates to new functionality Release note (bug fix): Fixed a bug from 21.1-alpha where a node decommissioning process could sometimes hang or fail when the decommission request was submitted via the node being decommissioned. --- pkg/cli/node.go | 9 +-- pkg/cmd/roachtest/acceptance.go | 10 +-- .../roachtest/mixed_version_decommission.go | 4 +- pkg/server/BUILD.bazel | 1 + pkg/server/admin.go | 21 ++++-- pkg/server/admin_test.go | 73 +++++++++++++++++++ pkg/server/connectivity_test.go | 29 ++++++-- pkg/server/server.go | 13 ++++ 8 files changed, 129 insertions(+), 31 deletions(-) diff --git a/pkg/cli/node.go b/pkg/cli/node.go index a594bf5595c3..1eb6ba9dfb8b 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -466,18 +466,11 @@ func runDecommissionNodeImpl( NodeIDs: nodeIDs, TargetMembership: livenesspb.MembershipStatus_DECOMMISSIONED, } - resp, err := c.Decommission(ctx, req) + _, err = c.Decommission(ctx, req) if err != nil { fmt.Fprintln(stderr) return errors.Wrap(err, "while trying to mark as decommissioned") } - if !reflect.DeepEqual(&prevResponse, resp) { - fmt.Fprintln(stderr) - if err := printDecommissionStatus(*resp); err != nil { - return err - } - prevResponse = *resp - } fmt.Fprintln(os.Stdout, "\nNo more data reported on target nodes. "+ "Please verify cluster health before removing the nodes.") diff --git a/pkg/cmd/roachtest/acceptance.go b/pkg/cmd/roachtest/acceptance.go index ff4be949bbbe..b78ab2b4c8d7 100644 --- a/pkg/cmd/roachtest/acceptance.go +++ b/pkg/cmd/roachtest/acceptance.go @@ -39,15 +39,7 @@ func registerAcceptance(r *testRegistry) { {name: "build-analyze", fn: runBuildAnalyze}, {name: "cli/node-status", fn: runCLINodeStatus}, {name: "cluster-init", fn: runClusterInit}, - {name: "decommission-self", - fn: runDecommissionSelf, - // Decommissioning self was observed to hang, though not in this test - // when run locally. More investigation is needed; there is a small - // chance that the original observation was in error. However, it - // seems likely that the problem exists even if it is rarely reproduced, - // so this test is skipped. - skip: "https://github.com/cockroachdb/cockroach/issues/56718", - }, + {name: "decommission-self", fn: runDecommissionSelf}, {name: "event-log", fn: runEventLog}, {name: "gossip/peerings", fn: runGossipPeerings}, {name: "gossip/restart", fn: runGossipRestart}, diff --git a/pkg/cmd/roachtest/mixed_version_decommission.go b/pkg/cmd/roachtest/mixed_version_decommission.go index 9ad0841a956d..aed72d9c93a9 100644 --- a/pkg/cmd/roachtest/mixed_version_decommission.go +++ b/pkg/cmd/roachtest/mixed_version_decommission.go @@ -87,9 +87,7 @@ func runDecommissionMixedVersions( // to communicate with the cluster (i.e. most commands against it will fail). // This is also why we're making sure to avoid decommissioning pinnedUpgrade // itself, as we use it to check the membership after. - // - // NB: we avoid runNode == targetNode here to temporarily avoid #56718. - fullyDecommissionStep(h.getRandNodeOtherThan(pinnedUpgrade), pinnedUpgrade, ""), + fullyDecommissionStep(h.getRandNodeOtherThan(pinnedUpgrade), h.getRandNode(), ""), checkOneMembership(pinnedUpgrade, "decommissioned"), ) diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 0616a5c9bf47..d30e408d8486 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -288,6 +288,7 @@ go_test( "//pkg/kv/kvserver/closedts/ctpb", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/kvserverpb", + "//pkg/kv/kvserver/liveness", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/roachpb", "//pkg/rpc", diff --git a/pkg/server/admin.go b/pkg/server/admin.go index e9c8cba606db..c00c482cddf4 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -1922,14 +1922,14 @@ func (s *adminServer) DecommissionStatus( ) (*serverpb.DecommissionStatusResponse, error) { // Get the number of replicas on each node. We *may* not need all of them, // but that would be more complicated than seems worth it right now. - ns, err := s.server.status.Nodes(ctx, &serverpb.NodesRequest{}) - if err != nil { - return nil, errors.Wrap(err, "loading node statuses") - } - nodeIDs := req.NodeIDs + // If no nodeIDs given, use all nodes. if len(nodeIDs) == 0 { + ns, err := s.server.status.Nodes(ctx, &serverpb.NodesRequest{}) + if err != nil { + return nil, errors.Wrap(err, "loading node statuses") + } for _, status := range ns.Nodes { nodeIDs = append(nodeIDs, status.Desc.NodeID) } @@ -1991,6 +1991,9 @@ func (s *adminServer) DecommissionStatus( } // Decommission sets the decommission flag to the specified value on the specified node(s). +// When the flag is set to DECOMMISSIONED, an empty response is returned on success -- this +// ensures a node can decommission itself, since the node could otherwise lose RPC access +// to the cluster while building the full response. func (s *adminServer) Decommission( ctx context.Context, req *serverpb.DecommissionRequest, ) (*serverpb.DecommissionStatusResponse, error) { @@ -2005,6 +2008,14 @@ func (s *adminServer) Decommission( if err := s.server.Decommission(ctx, req.TargetMembership, nodeIDs); err != nil { return nil, err } + + // We return an empty response when setting the final DECOMMISSIONED state, + // since a node can be asked to decommission itself which may cause it to + // lose access to cluster RPC and fail to populate the response. + if req.TargetMembership == livenesspb.MembershipStatus_DECOMMISSIONED { + return &serverpb.DecommissionStatusResponse{}, nil + } + return s.DecommissionStatus(ctx, &serverpb.DecommissionStatusRequest{NodeIDs: nodeIDs}) } diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go index 6b0844faf74d..2ccd9f305387 100644 --- a/pkg/server/admin_test.go +++ b/pkg/server/admin_test.go @@ -34,7 +34,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/rpc" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/debug" "github.com/cockroachdb/cockroach/pkg/server/serverpb" @@ -2010,3 +2013,73 @@ func TestEndpointTelemetryBasic(t *testing.T) { "/cockroach.server.serverpb.Status/Statements", ))) } + +func TestDecommissionSelf(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + skip.UnderRace(t) // can't handle 7-node clusters + + // Set up test cluster. + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 7, base.TestClusterArgs{ + ReplicationMode: base.ReplicationManual, // saves time + }) + defer tc.Stopper().Stop(ctx) + + // Decommission several nodes, including the node we're submitting the + // decommission request to. We use the admin client in order to test the + // admin server's logic, which involves a subsequent DecommissionStatus + // call which could fail if used from a node that's just decommissioned. + adminSrv := tc.Server(4) + conn, err := adminSrv.RPCContext().GRPCDialNode( + adminSrv.RPCAddr(), adminSrv.NodeID(), rpc.DefaultClass).Connect(ctx) + require.NoError(t, err) + adminClient := serverpb.NewAdminClient(conn) + decomNodeIDs := []roachpb.NodeID{ + tc.Server(4).NodeID(), + tc.Server(5).NodeID(), + tc.Server(6).NodeID(), + } + + // The DECOMMISSIONING call should return a full status response. + resp, err := adminClient.Decommission(ctx, &serverpb.DecommissionRequest{ + NodeIDs: decomNodeIDs, + TargetMembership: livenesspb.MembershipStatus_DECOMMISSIONING, + }) + require.NoError(t, err) + require.Len(t, resp.Status, len(decomNodeIDs)) + for i, nodeID := range decomNodeIDs { + status := resp.Status[i] + require.Equal(t, nodeID, status.NodeID) + // Liveness entries may not have been updated yet. + require.Contains(t, []livenesspb.MembershipStatus{ + livenesspb.MembershipStatus_ACTIVE, + livenesspb.MembershipStatus_DECOMMISSIONING, + }, status.Membership, "unexpected membership status %v for node %v", status, nodeID) + } + + // The DECOMMISSIONED call should return an empty response, to avoid + // erroring due to loss of cluster RPC access when decommissioning self. + resp, err = adminClient.Decommission(ctx, &serverpb.DecommissionRequest{ + NodeIDs: decomNodeIDs, + TargetMembership: livenesspb.MembershipStatus_DECOMMISSIONED, + }) + require.NoError(t, err) + require.Empty(t, resp.Status) + + // The nodes should now have been (or soon become) decommissioned. + for i := 0; i < tc.NumServers(); i++ { + srv := tc.Server(i) + expect := livenesspb.MembershipStatus_ACTIVE + for _, nodeID := range decomNodeIDs { + if srv.NodeID() == nodeID { + expect = livenesspb.MembershipStatus_DECOMMISSIONED + break + } + } + require.Eventually(t, func() bool { + liveness, ok := srv.NodeLiveness().(*liveness.NodeLiveness).GetLiveness(srv.NodeID()) + return ok && liveness.Membership == expect + }, 5*time.Second, 100*time.Millisecond, "timed out waiting for node %v status %v", i, expect) + } +} diff --git a/pkg/server/connectivity_test.go b/pkg/server/connectivity_test.go index a167ed9a56a6..dc95508ab374 100644 --- a/pkg/server/connectivity_test.go +++ b/pkg/server/connectivity_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/rpc" @@ -352,24 +353,28 @@ func TestDecommissionedNodeCannotConnect(t *testing.T) { tc := testcluster.StartTestCluster(t, numNodes, tcArgs) defer tc.Stopper().Stop(ctx) + scratchKey := tc.ScratchRange(t) + scratchRange := tc.LookupRangeOrFatal(t, scratchKey) + require.Len(t, scratchRange.InternalReplicas, 1) + require.Equal(t, tc.Server(0).NodeID(), scratchRange.InternalReplicas[0].NodeID) + + decomSrv := tc.Server(2) for _, status := range []livenesspb.MembershipStatus{ livenesspb.MembershipStatus_DECOMMISSIONING, livenesspb.MembershipStatus_DECOMMISSIONED, } { - require.NoError(t, tc.Servers[0].Decommission(ctx, status, []roachpb.NodeID{3})) + require.NoError(t, tc.Servers[0].Decommission(ctx, status, []roachpb.NodeID{decomSrv.NodeID()})) } testutils.SucceedsSoon(t, func() error { for _, idx := range []int{0, 1} { clusterSrv := tc.Server(idx) - decomSrv := tc.Server(2) // Within a short period of time, the cluster (n1, n2) will refuse to reach out to n3. { _, err := clusterSrv.RPCContext().GRPCDialNode( decomSrv.RPCAddr(), decomSrv.NodeID(), rpc.DefaultClass, ).Connect(ctx) - cause := errors.UnwrapAll(err) - s, ok := grpcstatus.FromError(cause) + s, ok := grpcstatus.FromError(errors.UnwrapAll(err)) if !ok || s.Code() != codes.PermissionDenied { return errors.Errorf("expected permission denied for n%d->n%d, got %v", clusterSrv.NodeID(), decomSrv.NodeID(), err) } @@ -380,8 +385,7 @@ func TestDecommissionedNodeCannotConnect(t *testing.T) { _, err := decomSrv.RPCContext().GRPCDialNode( clusterSrv.RPCAddr(), decomSrv.NodeID(), rpc.DefaultClass, ).Connect(ctx) - cause := errors.UnwrapAll(err) - s, ok := grpcstatus.FromError(cause) + s, ok := grpcstatus.FromError(errors.UnwrapAll(err)) if !ok || s.Code() != codes.PermissionDenied { return errors.Errorf("expected permission denied for n%d->n%d, got %v", decomSrv.NodeID(), clusterSrv.NodeID(), err) } @@ -389,4 +393,17 @@ func TestDecommissionedNodeCannotConnect(t *testing.T) { } return nil }) + + // Trying to scan the scratch range via the decommissioned node should + // now result in a permission denied error. + _, err := decomSrv.DB().Scan(ctx, scratchKey, keys.MaxKey, 1) + require.Error(t, err) + + // TODO(erikgrinaker): until cockroachdb/errors preserves grpcstatus.Error + // across errors.EncodeError() we just check for any error, this should + // check that it matches codes.PermissionDenied later. + //err = errors.UnwrapAll(err) + //s, ok := grpcstatus.FromError(err) + //require.True(t, ok, "expected gRPC error, got %T (%v)", err, err) + //require.Equal(t, codes.PermissionDenied, s.Code(), "expected permission denied error") } diff --git a/pkg/server/server.go b/pkg/server/server.go index 4772888358b6..87fcc0f5c9a8 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -23,6 +23,7 @@ import ( "path/filepath" "reflect" "runtime" + "sort" "strconv" "strings" "sync" @@ -2129,6 +2130,18 @@ func (s *Server) Decommission( } } + // If we're asked to decommission ourself we may lose access to cluster RPC, + // so we decommission ourself last. We copy the slice to avoid mutating the + // input slice. + if targetStatus == livenesspb.MembershipStatus_DECOMMISSIONED { + orderedNodeIDs := make([]roachpb.NodeID, len(nodeIDs)) + copy(orderedNodeIDs, nodeIDs) + sort.SliceStable(orderedNodeIDs, func(i, j int) bool { + return orderedNodeIDs[j] == s.NodeID() + }) + nodeIDs = orderedNodeIDs + } + var event eventpb.EventPayload var nodeDetails *eventpb.CommonNodeDecommissionDetails if targetStatus.Decommissioning() { From b2343025e704f22a6fd090b8c43cee9f431589a6 Mon Sep 17 00:00:00 2001 From: richardjcai Date: Tue, 2 Mar 2021 12:43:14 -0500 Subject: [PATCH 6/9] sql: add is_temporary and is_virtual columns to crdb_internal.create_statements These columns are useful for filtering for generating output for SHOW CREATE ALL TABLES. Release justification: None, change to internal table. Release note: None --- pkg/sql/crdb_internal.go | 6 +++- .../testdata/logic_test/crdb_internal | 33 +++++++++++++++++-- .../testdata/logic_test/crdb_internal_tenant | 4 +-- .../testdata/logic_test/create_statements | 8 +++-- .../logictest/testdata/logic_test/sequences | 6 ++-- 5 files changed, 47 insertions(+), 10 deletions(-) diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 792ce0b51f37..1b8156409f8d 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1935,7 +1935,9 @@ CREATE TABLE crdb_internal.create_statements ( alter_statements STRING[] NOT NULL, validate_statements STRING[] NOT NULL, has_partitions BOOL NOT NULL, - is_multi_region BOOL NOT NULL, + is_multi_region BOOL NOT NULL, + is_virtual BOOL NOT NULL, + is_temporary BOOL NOT NULL, INDEX(descriptor_id) ) `, virtualCurrentDB, false, /* includesIndexEntries */ @@ -2004,6 +2006,8 @@ CREATE TABLE crdb_internal.create_statements ( validateStmts, tree.MakeDBool(tree.DBool(hasPartitions)), tree.MakeDBool(tree.DBool(db != nil && db.IsMultiRegion())), + tree.MakeDBool(tree.DBool(table.IsVirtualTable())), + tree.MakeDBool(tree.DBool(table.IsTemporary())), ) }) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index ebee180adeb6..cbcf98546aa6 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -226,10 +226,10 @@ SELECT * FROM crdb_internal.builtin_functions WHERE function = '' ---- function signature category details -query ITTITTTTTTTBB colnames +query ITTITTTTTTTBBBB colnames SELECT * FROM crdb_internal.create_statements WHERE database_name = '' ---- -database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region +database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region is_virtual is_temporary query ITITTBTB colnames SELECT * FROM crdb_internal.table_columns WHERE descriptor_name = '' @@ -825,3 +825,32 @@ testdb root ALL statement ok SET DATABASE = test + +# Test crdb_internal.create_statements functionality. + +statement ok +CREATE TABLE normal_table() + +query B +SELECT is_virtual FROM crdb_internal.create_statements WHERE descriptor_name = 'normal_table' +---- +false + +query B +SELECT is_virtual FROM crdb_internal.create_statements WHERE descriptor_name = 'pg_views' +---- +true + +query B +SELECT is_temporary FROM crdb_internal.create_statements WHERE descriptor_name = 'normal_table' +---- +false + +statement ok +SET experimental_enable_temp_tables = 'on'; +CREATE TEMPORARY TABLE temp() + +query B +SELECT is_temporary FROM crdb_internal.create_statements WHERE descriptor_name = 'temp' +---- +true diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index cf9b51496429..3989a10eea66 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -238,10 +238,10 @@ SELECT * FROM crdb_internal.builtin_functions WHERE function = '' ---- function signature category details -query ITTITTTTTTTBB colnames +query ITTITTTTTTTBBBB colnames SELECT * FROM crdb_internal.create_statements WHERE database_name = '' ---- -database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region +database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region is_virtual is_temporary query ITITTBTB colnames SELECT * FROM crdb_internal.table_columns WHERE descriptor_name = '' diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 9373e9868649..ffba1ef833c5 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -190,7 +190,9 @@ CREATE TABLE crdb_internal.create_statements ( validate_statements STRING[] NOT NULL, has_partitions BOOL NOT NULL, is_multi_region BOOL NOT NULL, - INDEX create_statements_descriptor_id_idx (descriptor_id ASC) STORING (database_id, database_name, schema_name, descriptor_type, descriptor_name, create_statement, state, create_nofks, alter_statements, validate_statements, has_partitions, is_multi_region) + is_virtual BOOL NOT NULL, + is_temporary BOOL NOT NULL, + INDEX create_statements_descriptor_id_idx (descriptor_id ASC) STORING (database_id, database_name, schema_name, descriptor_type, descriptor_name, create_statement, state, create_nofks, alter_statements, validate_statements, has_partitions, is_multi_region, is_virtual, is_temporary) ) CREATE TABLE crdb_internal.create_statements ( database_id INT8 NULL, database_name STRING NULL, @@ -205,7 +207,9 @@ CREATE TABLE crdb_internal.create_statements ( validate_statements STRING[] NOT NULL, has_partitions BOOL NOT NULL, is_multi_region BOOL NOT NULL, - INDEX create_statements_descriptor_id_idx (descriptor_id ASC) STORING (database_id, database_name, schema_name, descriptor_type, descriptor_name, create_statement, state, create_nofks, alter_statements, validate_statements, has_partitions, is_multi_region) + is_virtual BOOL NOT NULL, + is_temporary BOOL NOT NULL, + INDEX create_statements_descriptor_id_idx (descriptor_id ASC) STORING (database_id, database_name, schema_name, descriptor_type, descriptor_name, create_statement, state, create_nofks, alter_statements, validate_statements, has_partitions, is_multi_region, is_virtual, is_temporary) ) {} {} CREATE TABLE crdb_internal.create_type_statements ( database_id INT8 NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/sequences b/pkg/sql/logictest/testdata/logic_test/sequences index ffc988e39351..4c69588d3257 100644 --- a/pkg/sql/logictest/testdata/logic_test/sequences +++ b/pkg/sql/logictest/testdata/logic_test/sequences @@ -133,11 +133,11 @@ CREATE SEQUENCE ignored_options_test NO CYCLE statement ok CREATE SEQUENCE show_create_test -query ITTITTTTTTTBB colnames +query ITTITTTTTTTBBBB colnames SELECT * FROM crdb_internal.create_statements WHERE descriptor_name = 'show_create_test' ---- -database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region -52 test public 63 sequence show_create_test CREATE SEQUENCE public.show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 PUBLIC CREATE SEQUENCE public.show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 {} {} false false +database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region is_virtual is_temporary +52 test public 63 sequence show_create_test CREATE SEQUENCE public.show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 PUBLIC CREATE SEQUENCE public.show_create_test MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1 {} {} false false false false query TT colnames SHOW CREATE SEQUENCE show_create_test From 83a01b6e5c5f9ddf3da73409de62aed558dd3e4e Mon Sep 17 00:00:00 2001 From: richardjcai Date: Wed, 3 Mar 2021 11:27:35 -0500 Subject: [PATCH 7/9] sql: disallow GRANT/REVOKE on system tables Disallow GRANT/REVOKE operations on system objects to avoid potential deadlocks related to version bumps. Release justification: bug fix Release note (sql change): Disallow `GRANT/REVOKE` operations on system tables. Release note (bug fix): Fix a bug where `GRANT/REVOKE` on the `system.lease` table would result in a deadlock. --- pkg/sql/grant_revoke.go | 12 ++++ .../logictest/testdata/logic_test/grant_table | 8 +++ pkg/sql/logictest/testdata/logic_test/system | 68 +++++++++---------- pkg/sql/logictest/testdata/logic_test/user | 7 +- 4 files changed, 58 insertions(+), 37 deletions(-) diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go index 63528be6ffc6..2f6d57ff442c 100644 --- a/pkg/sql/grant_revoke.go +++ b/pkg/sql/grant_revoke.go @@ -14,6 +14,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc" @@ -21,6 +22,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -185,6 +188,15 @@ func (n *changePrivilegesNode) startExec(params runParams) error { // we update them in KV below. b := p.txn.NewBatch() for _, descriptor := range descriptors { + // Disallow privilege changes on system objects. For more context, see #43842. + op := "REVOKE" + if n.isGrant { + op = "GRANT" + } + if descriptor.GetID() < keys.MinUserDescID { + return pgerror.Newf(pgcode.InsufficientPrivilege, "cannot %s on system object", op) + } + if err := p.CheckPrivilege(ctx, descriptor, privilege.GRANT); err != nil { return err } diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 684d9175eb97..0455ae381bed 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -1922,3 +1922,11 @@ b public t2 root ALL statement error pq: invalid privilege type USAGE for table GRANT USAGE ON t TO testuser + +# Grant / Revoke should not work on system tables. + +statement error pq: cannot GRANT on system object +GRANT SELECT ON system.lease TO testuser + +statement error pq: cannot REVOKE on system object +REVOKE SELECT ON system.lease FROM testuser diff --git a/pkg/sql/logictest/testdata/logic_test/system b/pkg/sql/logictest/testdata/logic_test/system index dbd2f8adcb78..85321ca55eb8 100644 --- a/pkg/sql/logictest/testdata/logic_test/system +++ b/pkg/sql/logictest/testdata/logic_test/system @@ -468,110 +468,110 @@ DROP DATABASE system statement error user root does not have DROP privilege on relation users DROP TABLE system.users -statement error user root does not have ALL privilege on database system +statement error pq: cannot GRANT on system object GRANT ALL ON DATABASE system TO testuser -statement error user root does not have INSERT privilege on database system +statement error pq: cannot GRANT on system object GRANT GRANT, SELECT, INSERT ON DATABASE system TO testuser -statement ok +statement error pq: cannot GRANT on system object GRANT GRANT, SELECT ON DATABASE system TO testuser -statement error user root does not have ALL privilege on relation namespace +statement error pq: cannot GRANT on system object GRANT ALL ON system.namespace TO testuser -statement error user root does not have INSERT privilege on relation namespace +statement error pq: cannot GRANT on system object GRANT GRANT, SELECT, INSERT ON system.namespace TO testuser -statement ok +statement error pq: cannot GRANT on system object GRANT GRANT, SELECT ON system.namespace TO testuser -statement ok +statement error pq: cannot GRANT on system object GRANT SELECT ON system.descriptor TO testuser # Superusers must have exactly the allowed privileges. -statement error user root does not have ALL privilege on database system +statement error pq: cannot GRANT on system object GRANT ALL ON DATABASE system TO root -statement error user root does not have DELETE privilege on database system +statement error pq: cannot GRANT on system object GRANT DELETE, INSERT ON DATABASE system TO root -statement error user root does not have ALL privilege on relation namespace +statement error pq: cannot GRANT on system object GRANT ALL ON system.namespace TO root -statement error user root does not have DELETE privilege on relation descriptor +statement error pq: cannot GRANT on system object GRANT DELETE, INSERT ON system.descriptor TO root -statement error user root does not have ALL privilege on relation descriptor +statement error pq: cannot GRANT on system object GRANT ALL ON system.descriptor TO root -statement error user root must have exactly GRANT, SELECT privileges on system database with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON DATABASE system FROM root -statement error user root must have exactly GRANT, SELECT privileges on system table with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON system.namespace FROM root -statement error user root does not have ALL privilege on relation namespace +statement error pq: cannot REVOKE on system object REVOKE ALL ON system.namespace FROM root -statement error user root does not have privileges over system table with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT,SELECT ON system.namespace FROM root -statement error user root does not have ALL privilege on database system +statement error pq: cannot GRANT on system object GRANT ALL ON DATABASE system TO admin -statement error user root does not have DELETE privilege on database system +statement error pq: cannot GRANT on system object GRANT DELETE, INSERT ON DATABASE system TO admin -statement error user admin must have exactly GRANT, SELECT privileges on system database with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON DATABASE system FROM admin -statement error user root does not have ALL privilege on relation namespace +statement error pq: cannot GRANT on system object GRANT ALL ON system.namespace TO admin -statement error user root does not have DELETE privilege on relation descriptor +statement error pq: cannot GRANT on system object GRANT DELETE, INSERT ON system.descriptor TO admin -statement error user root does not have ALL privilege on relation descriptor +statement error pq: cannot GRANT on system object GRANT ALL ON system.descriptor TO admin -statement error user admin must have exactly GRANT, SELECT privileges on system table with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON system.descriptor FROM admin -statement error user admin must have exactly GRANT, SELECT privileges on system database with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON DATABASE system FROM admin -statement error user admin must have exactly GRANT, SELECT privileges on system table with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT ON system.namespace FROM admin -statement error user root does not have ALL privilege on relation namespace +statement error pq: cannot REVOKE on system object REVOKE ALL ON system.namespace FROM admin -statement error user admin does not have privileges over system table with ID=.* +statement error pq: cannot REVOKE on system object REVOKE GRANT,SELECT ON system.namespace FROM admin # Some tables (we test system.lease here) used to allow multiple privilege sets for # backwards compatibility, and superusers were allowed very wide privileges. # We make sure this is no longer the case. -statement error user root does not have ALL privilege on relation lease +statement error pq: cannot GRANT on system object GRANT ALL ON system.lease TO testuser -statement error user root does not have CREATE privilege on relation lease +statement error pq: cannot GRANT on system object GRANT CREATE on system.lease to root -statement error user root does not have CREATE privilege on relation lease +statement error pq: cannot GRANT on system object GRANT CREATE on system.lease to admin -statement error user root does not have CREATE privilege on relation lease +statement error pq: cannot GRANT on system object GRANT CREATE on system.lease to testuser -statement error user root does not have ALL privilege on relation lease +statement error pq: cannot GRANT on system object GRANT ALL ON system.lease TO root -statement error user root does not have ALL privilege on relation lease +statement error pq: cannot GRANT on system object GRANT ALL ON system.lease TO admin -statement error user root does not have ALL privilege on relation lease +statement error pq: cannot GRANT on system object GRANT ALL ON system.lease TO testuser # NB: the "order by" is necessary or this test is flaky under DistSQL. diff --git a/pkg/sql/logictest/testdata/logic_test/user b/pkg/sql/logictest/testdata/logic_test/user index bd0283c10555..0351355f0128 100644 --- a/pkg/sql/logictest/testdata/logic_test/user +++ b/pkg/sql/logictest/testdata/logic_test/user @@ -137,9 +137,6 @@ root statement ok ALTER USER testuser CREATEROLE -statement ok -GRANT SELECT ON system.role_options to testuser - user testuser statement ok @@ -148,6 +145,8 @@ CREATE ROLE user4 CREATEROLE statement ok CREATE USER user5 NOLOGIN +user root + query TTT SELECT * FROM system.role_options ---- @@ -156,6 +155,8 @@ user4 CREATEROLE NULL user4 NOLOGIN NULL user5 NOLOGIN NULL +user testuser + statement ok DROP ROLE user4 From 099e8431172600b47af79c7c8d07e71c621520cc Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 4 Mar 2021 09:59:37 +1100 Subject: [PATCH 8/9] geomfn: prevent smaller densifyFrac for ST_{Frechet,Hausdorff}Distance Release justification: low-risk change to existing functionality Release note (sql change): Prevent densifyFracs < 1e-6 for ST_FrechetDistance and ST_HausdorffDistance to protect panics and out of memory errors. --- pkg/geo/geomfn/distance.go | 27 +++++++++++++++++++-------- pkg/geo/geomfn/distance_test.go | 10 +++------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/pkg/geo/geomfn/distance.go b/pkg/geo/geomfn/distance.go index e888e58f86f1..59bf0112f7d8 100644 --- a/pkg/geo/geomfn/distance.go +++ b/pkg/geo/geomfn/distance.go @@ -739,14 +739,8 @@ func FrechetDistanceDensify(a, b geo.Geometry, densifyFrac float64) (*float64, e if a.SRID() != b.SRID() { return nil, geo.NewMismatchingSRIDsError(a.SpatialObject(), b.SpatialObject()) } - // GEOS throws SIGFPE due to division by zero if densifyFraq is too small, - // so we explicitly error instead. The threshold was empirically found to be - // 1e-20, while 1e-19 results in "geos error: vector". - // - // Note that small values of densifyFrac are prohibitively expensive and - // likely to cause out-of-memory conditions. - if densifyFrac < 1e-19 { - return nil, errors.New("densifyFrac is too small") + if err := verifyDensifyFrac(densifyFrac); err != nil { + return nil, err } distance, err := geos.FrechetDistanceDensify(a.EWKB(), b.EWKB(), densifyFrac) if err != nil { @@ -778,6 +772,10 @@ func HausdorffDistanceDensify(a, b geo.Geometry, densifyFrac float64) (*float64, if a.SRID() != b.SRID() { return nil, geo.NewMismatchingSRIDsError(a.SpatialObject(), b.SpatialObject()) } + if err := verifyDensifyFrac(densifyFrac); err != nil { + return nil, err + } + distance, err := geos.HausdorffDistanceDensify(a.EWKB(), b.EWKB(), densifyFrac) if err != nil { return nil, err @@ -803,3 +801,16 @@ func ClosestPoint(a, b geo.Geometry) (geo.Geometry, error) { } return closestPoint, nil } + +func verifyDensifyFrac(f float64) error { + if f < 0 || f > 1 { + return errors.Newf("fraction must be in range [0, 1], got %f", f) + } + // Very small densifyFrac potentially causes a SIGFPE or generate a large + // amount of memory. Guard against this. + const fracTooSmall = 1e-6 + if f > 0 && f < fracTooSmall { + return errors.Newf("fraction %f is too small, must be at least %f", f, fracTooSmall) + } + return nil +} diff --git a/pkg/geo/geomfn/distance_test.go b/pkg/geo/geomfn/distance_test.go index 0585dd10bdf9..9079dca834ab 100644 --- a/pkg/geo/geomfn/distance_test.go +++ b/pkg/geo/geomfn/distance_test.go @@ -790,11 +790,8 @@ func TestFrechetDistanceDensify(t *testing.T) { densifyFrac float64 }{ // Very small densifyFrac causes a SIGFPE in GEOS due to division-by-zero. - // The threshold was empirically found to be at 1e-20, while at 1e-19 - // GEOS instead returns the error "geos error: vector". We explicitly - // disallow <1e-19 in the code, and test that both of these error. We do - // not test larger values, since very small densify values consume a - // large amount of memory causing out-of-memory errors. + // We explicitly disallow <1e-6 in the code, and test that both of these error. + {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-7}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-19}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-20}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-100}, @@ -867,8 +864,6 @@ func TestHausdorffDistanceDensify(t *testing.T) { {"LINESTRING EMPTY", "LINESTRING EMPTY", 0.5, nil}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING EMPTY", 0.5, nil}, {"LINESTRING EMPTY", "LINESTRING (10 10, 10 150, 130 10)", 0.5, nil}, - {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-100, pf(14.142135623730951)}, - {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1e-20, pf(14.142135623730951)}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 0.2, pf(66)}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 0.4, pf(56.66666666666667)}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 0.6, pf(70)}, @@ -902,6 +897,7 @@ func TestHausdorffDistanceDensify(t *testing.T) { {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", -1}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", -0.1}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 0.0}, + {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 0.0000001}, {"LINESTRING (130 0, 0 0, 0 150)", "LINESTRING (10 10, 10 150, 130 10)", 1.1}, } From df8a5ed5b67ef96758de50be623eba4679dbfe27 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 4 Mar 2021 10:18:33 +1100 Subject: [PATCH 9/9] sql: set zone configs before backfill on ALTER TABLE LOCALITY/PRIMARY KEY Ensure that the zone configurations are correctly set before the backfill runs, so data is backfilled to the correct location upfront for REGIONAL BY ROW tables. Note this bug also exists on existing ALTER PRIMARY KEY statements. Release justification: bug fixes and low-risk updates to new functionality Release note (bug fix): Fix bug where zone configurations on indexes were not copied before the backfill of an ALTER PRIMARY KEY. They used to be copied afterwards instead. --- pkg/ccl/multiregionccl/BUILD.bazel | 1 + .../multiregionccl/regional_by_row_test.go | 146 +++++++++++++- pkg/ccl/partitionccl/BUILD.bazel | 2 + .../partitionccl/alter_primary_key_test.go | 51 +++++ pkg/ccl/testutilsccl/BUILD.bazel | 19 ++ pkg/ccl/testutilsccl/alter_primary_key.go | 118 +++++++++++ pkg/sql/region_util.go | 2 +- pkg/sql/schema_changer.go | 187 +++++++++++------- 8 files changed, 444 insertions(+), 82 deletions(-) create mode 100644 pkg/ccl/partitionccl/alter_primary_key_test.go create mode 100644 pkg/ccl/testutilsccl/BUILD.bazel create mode 100644 pkg/ccl/testutilsccl/alter_primary_key.go diff --git a/pkg/ccl/multiregionccl/BUILD.bazel b/pkg/ccl/multiregionccl/BUILD.bazel index db22ed05a109..71faebff83ab 100644 --- a/pkg/ccl/multiregionccl/BUILD.bazel +++ b/pkg/ccl/multiregionccl/BUILD.bazel @@ -31,6 +31,7 @@ go_test( "//pkg/base", "//pkg/ccl/multiregionccl/multiregionccltestutils", "//pkg/ccl/partitionccl", + "//pkg/ccl/testutilsccl", "//pkg/ccl/utilccl", "//pkg/jobs", "//pkg/keys", diff --git a/pkg/ccl/multiregionccl/regional_by_row_test.go b/pkg/ccl/multiregionccl/regional_by_row_test.go index da6192f89c5e..024bfdea4fe4 100644 --- a/pkg/ccl/multiregionccl/regional_by_row_test.go +++ b/pkg/ccl/multiregionccl/regional_by_row_test.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl/multiregionccltestutils" + "github.com/cockroachdb/cockroach/pkg/ccl/testutilsccl" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -34,6 +35,129 @@ import ( "github.com/stretchr/testify/require" ) +// TestAlterTableLocalityRegionalByRowCorrectZoneConfigBeforeBackfill tests that +// the zone configurations are properly set up before the LOCALITY REGIONAL BY ROW +// backfill begins. +func TestAlterTableLocalityRegionalByRowCorrectZoneConfigBeforeBackfill(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testCases := []testutilsccl.AlterPrimaryKeyCorrectZoneConfigTestCase{ + { + Desc: "REGIONAL BY TABLE to REGIONAL BY ROW", + SetupQuery: `CREATE TABLE t.test (k INT NOT NULL, v INT) LOCALITY REGIONAL BY TABLE`, + AlterQuery: `ALTER TABLE t.test SET LOCALITY REGIONAL BY ROW`, + ExpectedIntermediateZoneConfigs: []testutilsccl.AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig{ + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR TABLE t.test`, + ExpectedTarget: `DATABASE t`, + ExpectedSQL: `ALTER DATABASE t CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR PARTITION "ajstorm-1" OF INDEX t.test@new_primary_key`, + ExpectedTarget: `PARTITION "ajstorm-1" OF INDEX t.public.test@new_primary_key`, + ExpectedSQL: `ALTER PARTITION "ajstorm-1" OF INDEX t.public.test@new_primary_key CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + }, + }, + { + Desc: "GLOBAL to REGIONAL BY ROW", + SetupQuery: `CREATE TABLE t.test (k INT NOT NULL, v INT) LOCALITY GLOBAL`, + AlterQuery: `ALTER TABLE t.test SET LOCALITY REGIONAL BY ROW`, + ExpectedIntermediateZoneConfigs: []testutilsccl.AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig{ + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR TABLE t.test`, + ExpectedTarget: `TABLE t.public.test`, + ExpectedSQL: `ALTER TABLE t.public.test CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + global_reads = true, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR PARTITION "ajstorm-1" OF INDEX t.test@new_primary_key`, + ExpectedTarget: `PARTITION "ajstorm-1" OF INDEX t.public.test@new_primary_key`, + ExpectedSQL: `ALTER PARTITION "ajstorm-1" OF INDEX t.public.test@new_primary_key CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + }, + }, + { + Desc: "REGIONAL BY ROW to REGIONAL BY TABLE", + SetupQuery: `CREATE TABLE t.test (k INT NOT NULL, v INT) LOCALITY REGIONAL BY ROW`, + AlterQuery: `ALTER TABLE t.test SET LOCALITY REGIONAL BY TABLE`, + ExpectedIntermediateZoneConfigs: []testutilsccl.AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig{ + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR TABLE t.test`, + ExpectedTarget: `DATABASE t`, + ExpectedSQL: `ALTER DATABASE t CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + }, + }, + { + Desc: "REGIONAL BY ROW to GLOBAL", + SetupQuery: `CREATE TABLE t.test (k INT NOT NULL, v INT) LOCALITY REGIONAL BY ROW`, + AlterQuery: `ALTER TABLE t.test SET LOCALITY GLOBAL`, + ExpectedIntermediateZoneConfigs: []testutilsccl.AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig{ + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR TABLE t.test`, + ExpectedTarget: `DATABASE t`, + ExpectedSQL: `ALTER DATABASE t CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 90000, + num_replicas = 3, + num_voters = 3, + constraints = '{+region=ajstorm-1: 1}', + voter_constraints = '[+region=ajstorm-1]', + lease_preferences = '[[+region=ajstorm-1]]'`, + }, + }, + }, + } + testutilsccl.AlterPrimaryKeyCorrectZoneConfigTest( + t, + `CREATE DATABASE t PRIMARY REGION "ajstorm-1"`, + testCases, + ) +} + // TestAlterTableLocalityRegionalByRowError tests an alteration involving // REGIONAL BY ROW which gets its async job interrupted by some sort of // error or cancellation. After this, we expect the table to retain @@ -60,7 +184,11 @@ func TestAlterTableLocalityRegionalByRowError(t *testing.T) { ctx := context.Background() const showCreateTableStringSQL = `SELECT create_statement FROM [SHOW CREATE TABLE t.test]` - const showZoneConfigurationSQL = `SHOW ZONE CONFIGURATION FROM TABLE t.test` + const zoneConfigureSQLStatements = ` + SELECT coalesce(string_agg(raw_config_sql, ';' ORDER BY raw_config_sql), 'NULL') + FROM crdb_internal.zones + WHERE database_name = 't' AND table_name = 'test' + ` // alterState is a struct that contains an action for a base test case // to execute ALTER TABLE t.test SET LOCALITY against. @@ -262,7 +390,7 @@ USE t; require.Error(t, err) require.Contains(t, err.Error(), errorMode.errorContains) - // Grab a copy of SHOW CREATE TABLE and SHOW ZONE CONFIGURATION before we run + // Grab a copy of SHOW CREATE TABLE and zone configuration data before we run // any ALTER query. The result should match if the operation fails. var originalCreateTableOutput string require.NoError( @@ -270,10 +398,10 @@ USE t; sqlDB.QueryRow(showCreateTableStringSQL).Scan(&originalCreateTableOutput), ) - var originalTarget, originalZoneConfig string + var originalZoneConfig string require.NoError( t, - sqlDB.QueryRow(showZoneConfigurationSQL).Scan(&originalTarget, &originalZoneConfig), + sqlDB.QueryRow(zoneConfigureSQLStatements).Scan(&originalZoneConfig), ) // Ensure that the mutations corresponding to the primary key change are cleaned up and @@ -316,16 +444,14 @@ USE t; } // Ensure SHOW ZONE CONFIGURATION has not changed. - var target, zoneConfig string + var zoneConfig string require.NoError( t, - sqlDB.QueryRow(showZoneConfigurationSQL).Scan(&target, &zoneConfig), + sqlDB.QueryRow(zoneConfigureSQLStatements).Scan(&zoneConfig), ) - if !(target == originalTarget && zoneConfig == originalZoneConfig) { + if zoneConfig != originalZoneConfig { return errors.Errorf( - "expected zone configuration to not have changed, got %s/%s, sql %s/%s", - originalTarget, - target, + "expected zone configuration statements to not have changed, got %s, sql %s", originalZoneConfig, zoneConfig, ) diff --git a/pkg/ccl/partitionccl/BUILD.bazel b/pkg/ccl/partitionccl/BUILD.bazel index 5159a0c48a32..a52dc671a5c5 100644 --- a/pkg/ccl/partitionccl/BUILD.bazel +++ b/pkg/ccl/partitionccl/BUILD.bazel @@ -30,6 +30,7 @@ go_test( name = "partitionccl_test", size = "medium", srcs = [ + "alter_primary_key_test.go", "drop_test.go", "main_test.go", "partition_test.go", @@ -40,6 +41,7 @@ go_test( "//pkg/base", "//pkg/ccl/importccl", "//pkg/ccl/storageccl", + "//pkg/ccl/testutilsccl", "//pkg/ccl/utilccl", "//pkg/config", "//pkg/config/zonepb", diff --git a/pkg/ccl/partitionccl/alter_primary_key_test.go b/pkg/ccl/partitionccl/alter_primary_key_test.go new file mode 100644 index 000000000000..4e9896eaa387 --- /dev/null +++ b/pkg/ccl/partitionccl/alter_primary_key_test.go @@ -0,0 +1,51 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package partitionccl + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl/testutilsccl" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +func TestAlterPrimaryKeyCorrectZoneConfigBeforeBackfill(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testCases := []testutilsccl.AlterPrimaryKeyCorrectZoneConfigTestCase{ + { + Desc: "ALTER PRIMARY KEY", + SetupQuery: `CREATE TABLE t.test (k INT NOT NULL, v INT NOT NULL, INDEX v_idx (v)); +ALTER INDEX t.test@v_idx CONFIGURE ZONE USING gc.ttlseconds = 888 + `, + AlterQuery: `ALTER TABLE t.test ALTER PRIMARY KEY USING COLUMNS (v)`, + ExpectedIntermediateZoneConfigs: []testutilsccl.AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig{ + { + ShowConfigStatement: `SHOW ZONE CONFIGURATION FOR INDEX t.test@v_idx_rewrite_for_primary_key_change`, + ExpectedTarget: `INDEX t.public.test@v_idx_rewrite_for_primary_key_change`, + ExpectedSQL: `ALTER INDEX t.public.test@v_idx_rewrite_for_primary_key_change CONFIGURE ZONE USING + range_min_bytes = 134217728, + range_max_bytes = 536870912, + gc.ttlseconds = 888, + num_replicas = 1, + constraints = '[]', + lease_preferences = '[]'`, + }, + }, + }, + } + + testutilsccl.AlterPrimaryKeyCorrectZoneConfigTest( + t, + `CREATE DATABASE t`, + testCases, + ) +} diff --git a/pkg/ccl/testutilsccl/BUILD.bazel b/pkg/ccl/testutilsccl/BUILD.bazel new file mode 100644 index 000000000000..4cee750fe63a --- /dev/null +++ b/pkg/ccl/testutilsccl/BUILD.bazel @@ -0,0 +1,19 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "testutilsccl", + srcs = ["alter_primary_key.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/testutilsccl", + visibility = ["//visibility:public"], + deps = [ + "//pkg/base", + "//pkg/roachpb", + "//pkg/sql", + "//pkg/sql/execinfra", + "//pkg/sql/sqltestutils", + "//pkg/sql/tests", + "//pkg/testutils/serverutils", + "//pkg/util", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/ccl/testutilsccl/alter_primary_key.go b/pkg/ccl/testutilsccl/alter_primary_key.go new file mode 100644 index 000000000000..81c13bf18b4e --- /dev/null +++ b/pkg/ccl/testutilsccl/alter_primary_key.go @@ -0,0 +1,118 @@ +// Copyright 2021 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package testutilsccl + +import ( + "context" + gosql "database/sql" + "fmt" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/sqltestutils" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/stretchr/testify/require" +) + +// AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig is an expected +// intermediate zone configuration in the AlterPrimaryKeyCorrectZoneConfigTestCase. +type AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig struct { + ShowConfigStatement string + ExpectedTarget string + ExpectedSQL string +} + +// AlterPrimaryKeyCorrectZoneConfigTestCase is a test case for +// AlterPrimaryKeyCorrectZoneConfigTest. +type AlterPrimaryKeyCorrectZoneConfigTestCase struct { + Desc string + SetupQuery string + AlterQuery string + ExpectedIntermediateZoneConfigs []AlterPrimaryKeyCorrectZoneConfigIntermediateZoneConfig +} + +// AlterPrimaryKeyCorrectZoneConfigTest tests that zone configurations +// are correctly set before the backfill of a PRIMARY KEY. +func AlterPrimaryKeyCorrectZoneConfigTest( + t *testing.T, createDBStatement string, testCases []AlterPrimaryKeyCorrectZoneConfigTestCase, +) { + // Decrease the adopt loop interval so that retries happen quickly. + defer sqltestutils.SetTestJobsAdoptInterval()() + + chunkSize := int64(100) + maxValue := 4000 + + if util.RaceEnabled { + // Race builds are a lot slower, so use a smaller number of rows. + maxValue = 200 + chunkSize = 5 + } + + ctx := context.Background() + + for _, tc := range testCases { + t.Run(tc.Desc, func(t *testing.T) { + var db *gosql.DB + params, _ := tests.CreateTestServerParams() + params.Locality.Tiers = []roachpb.Tier{ + {Key: "region", Value: "ajstorm-1"}, + } + + runCheck := false + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + BackfillChunkSize: chunkSize, + }, + DistSQL: &execinfra.TestingKnobs{ + RunBeforeBackfillChunk: func(sp roachpb.Span) error { + if runCheck { + for _, subTC := range tc.ExpectedIntermediateZoneConfigs { + t.Run(subTC.ShowConfigStatement, func(t *testing.T) { + var target, sql string + require.NoError( + t, + db.QueryRow(subTC.ShowConfigStatement).Scan(&target, &sql), + ) + require.Equal(t, subTC.ExpectedTarget, target) + require.Equal(t, subTC.ExpectedSQL, sql) + }) + } + runCheck = false + } + return nil + }, + }, + } + s, sqlDB, _ := serverutils.StartServer(t, params) + db = sqlDB + defer s.Stopper().Stop(ctx) + + if _, err := sqlDB.Exec(fmt.Sprintf(` +%s; +USE t; +%s +`, createDBStatement, tc.SetupQuery)); err != nil { + t.Fatal(err) + } + + // Insert some rows so we can interrupt inspect state during backfill. + require.NoError(t, sqltestutils.BulkInsertIntoTable(sqlDB, maxValue)) + + runCheck = true + _, err := sqlDB.Exec(tc.AlterQuery) + require.NoError(t, err) + }) + } + +} diff --git a/pkg/sql/region_util.go b/pkg/sql/region_util.go index 0fd717a8f9c0..cbfebe01ba42 100644 --- a/pkg/sql/region_util.go +++ b/pkg/sql/region_util.go @@ -537,7 +537,7 @@ func ApplyZoneConfigForMultiRegionTable( if err != nil { return err } - var zoneConfig zonepb.ZoneConfig + zoneConfig := *zonepb.NewZoneConfig() if zoneRaw != nil { zoneConfig = *zoneRaw } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 0554c38731bb..a3a1cc038c8c 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -911,6 +911,15 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro if err != nil { return err } + _, dbDesc, err := descsCol.GetImmutableDatabaseByID( + ctx, + txn, + tbl.GetParentID(), + tree.DatabaseLookupFlags{Required: true}, + ) + if err != nil { + return err + } runStatus = "" // Apply mutations belonging to the same version. for i, mutation := range tbl.Mutations { @@ -946,6 +955,21 @@ func (sc *SchemaChanger) RunStateMachineBeforeBackfill(ctx context.Context) erro runStatus = RunningStatusDeleteOnly } } + // We might have to update some zone configs for indexes that are + // being rewritten. It is important that this is done _before_ the + // index swap occurs. The logic that generates spans for subzone + // configurations removes spans for indexes in the dropping state, + // which we don't want. So, set up the zone configs before we swap. + if err := sc.applyZoneConfigChangeForMutation( + ctx, + txn, + dbDesc, + tbl, + mutation, + false, // isDone + ); err != nil { + return err + } } if doNothing := runStatus == "" || tbl.Dropped(); doNothing { return nil @@ -1116,77 +1140,18 @@ func (sc *SchemaChanger) done(ctx context.Context) error { } } - // Some primary key change specific operations need to happen before - // and after the index swap occurs. - if pkSwap := mutation.GetPrimaryKeySwap(); pkSwap != nil { - // We might have to update some zone configs for indexes that are - // being rewritten. It is important that this is done _before_ the - // index swap occurs. The logic that generates spans for subzone - // configurations removes spans for indexes in the dropping state, - // which we don't want. So, set up the zone configs before we swap. - if lcSwap := pkSwap.LocalityConfigSwap; lcSwap != nil { - // We will add up to two options - one for the table itself, and one - // for all the new indexes associated with the table. - opts := make([]applyZoneConfigForMultiRegionTableOption, 0, 2) - - // For locality configs, we need to update the zone configs to match - // the new multi-region locality configuration, instead of - // copying the old zone configs over. - if mutation.Direction == descpb.DescriptorMutation_ADD { - opts = append( - opts, - applyZoneConfigForMultiRegionTableOptionTableNewConfig( - lcSwap.NewLocalityConfig, - ), - ) - switch lcSwap.NewLocalityConfig.Locality.(type) { - case *descpb.TableDescriptor_LocalityConfig_Global_, - *descpb.TableDescriptor_LocalityConfig_RegionalByTable_: - // Just the table re-writing the locality config change will suffice. - case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_: - // Apply new zone configurations for all newly partitioned indexes. - opts = append( - opts, - applyZoneConfigForMultiRegionTableOptionNewIndexes( - append( - []descpb.IndexID{pkSwap.NewPrimaryIndexId}, - pkSwap.NewIndexes..., - )..., - ), - ) - default: - return errors.AssertionFailedf( - "unknown locality on PK swap: %T", - lcSwap.NewLocalityConfig.Locality, - ) - } - } else { - // DROP is hit on cancellation, in which case we must roll back. - opts = append( - opts, - applyZoneConfigForMultiRegionTableOptionTableNewConfig( - lcSwap.OldLocalityConfig, - ), - ) - } - - if err := ApplyZoneConfigForMultiRegionTable( - ctx, - txn, - sc.execCfg, - *dbDesc.RegionConfig, - scTable, - opts..., - ); err != nil { - return err - } - } else { - // For the normal case, copy the zone configs over. - if err := maybeUpdateZoneConfigsForPKChange( - ctx, txn, sc.execCfg, scTable, pkSwap); err != nil { - return err - } - } + // Ensure that zone configurations are finalized (or rolled back) when + // done is called. + // This will configure the table zone config for multi-region transformations. + if err := sc.applyZoneConfigChangeForMutation( + ctx, + txn, + dbDesc, + scTable, + mutation, + true, // isDone + ); err != nil { + return err } // If we are refreshing a materialized view, then create GC jobs for all @@ -2457,6 +2422,86 @@ func (sc *SchemaChanger) queueCleanupJobs( return nil } +func (sc *SchemaChanger) applyZoneConfigChangeForMutation( + ctx context.Context, + txn *kv.Txn, + dbDesc *dbdesc.Immutable, + tableDesc *tabledesc.Mutable, + mutation descpb.DescriptorMutation, + isDone bool, +) error { + if pkSwap := mutation.GetPrimaryKeySwap(); pkSwap != nil { + if lcSwap := pkSwap.LocalityConfigSwap; lcSwap != nil { + // We will add up to two options - one for the table itself, and one + // for all the new indexes associated with the table. + opts := make([]applyZoneConfigForMultiRegionTableOption, 0, 2) + + // For locality configs, we need to update the zone configs to match + // the new multi-region locality configuration, instead of + // copying the old zone configs over. + if mutation.Direction == descpb.DescriptorMutation_ADD { + // Only apply the zone configuration on the table when the mutation + // is complete. + if isDone { + opts = append( + opts, + applyZoneConfigForMultiRegionTableOptionTableNewConfig( + lcSwap.NewLocalityConfig, + ), + ) + } + switch lcSwap.NewLocalityConfig.Locality.(type) { + case *descpb.TableDescriptor_LocalityConfig_Global_, + *descpb.TableDescriptor_LocalityConfig_RegionalByTable_: + // Nothing to do here. The table re-writing the locality config is all + // that is required. + case *descpb.TableDescriptor_LocalityConfig_RegionalByRow_: + // Apply new zone configurations for all newly partitioned indexes. + opts = append( + opts, + applyZoneConfigForMultiRegionTableOptionNewIndexes( + append( + []descpb.IndexID{pkSwap.NewPrimaryIndexId}, + pkSwap.NewIndexes..., + )..., + ), + ) + default: + return errors.AssertionFailedf( + "unknown locality on PK swap: %T", + lcSwap.NewLocalityConfig.Locality, + ) + } + } else { + // DROP is hit on cancellation, in which case we must roll back. + opts = append( + opts, + applyZoneConfigForMultiRegionTableOptionTableNewConfig( + lcSwap.OldLocalityConfig, + ), + ) + } + + return ApplyZoneConfigForMultiRegionTable( + ctx, + txn, + sc.execCfg, + *dbDesc.RegionConfig, + tableDesc, + opts..., + ) + } + + // For the plain ALTER PRIMARY KEY case, copy the zone configs over + // for any new indexes. + // Note this is done even for isDone = true, though not strictly necessary. + return maybeUpdateZoneConfigsForPKChange( + ctx, txn, sc.execCfg, tableDesc, pkSwap, + ) + } + return nil +} + // DeleteTableDescAndZoneConfig removes a table's descriptor and zone config from the KV database. func DeleteTableDescAndZoneConfig( ctx context.Context, db *kv.DB, codec keys.SQLCodec, tableDesc catalog.TableDescriptor,